diff --git a/airbyte-integrations/bases/base-java-s3/src/main/java/io/airbyte/integrations/destination/s3/avro/AvroRecordFactory.java b/airbyte-integrations/bases/base-java-s3/src/main/java/io/airbyte/integrations/destination/s3/avro/AvroRecordFactory.java index 6bc2fcec059d..e5ad1755fa57 100644 --- a/airbyte-integrations/bases/base-java-s3/src/main/java/io/airbyte/integrations/destination/s3/avro/AvroRecordFactory.java +++ b/airbyte-integrations/bases/base-java-s3/src/main/java/io/airbyte/integrations/destination/s3/avro/AvroRecordFactory.java @@ -11,6 +11,7 @@ import com.fasterxml.jackson.databind.node.ObjectNode; import io.airbyte.commons.jackson.MoreMappers; import io.airbyte.integrations.base.JavaBaseConstants; +import io.airbyte.integrations.base.TypingAndDedupingFlag; import io.airbyte.protocol.models.v0.AirbyteRecordMessage; import java.util.UUID; import org.apache.avro.Schema; @@ -32,8 +33,14 @@ public AvroRecordFactory(final Schema schema, final JsonAvroConverter converter) public GenericData.Record getAvroRecord(final UUID id, final AirbyteRecordMessage recordMessage) throws JsonProcessingException { final ObjectNode jsonRecord = MAPPER.createObjectNode(); - jsonRecord.put(JavaBaseConstants.COLUMN_NAME_AB_ID, id.toString()); - jsonRecord.put(JavaBaseConstants.COLUMN_NAME_EMITTED_AT, recordMessage.getEmittedAt()); + if (TypingAndDedupingFlag.isDestinationV2()) { + jsonRecord.put(JavaBaseConstants.COLUMN_NAME_AB_RAW_ID, id.toString()); + jsonRecord.put(JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT, recordMessage.getEmittedAt()); + jsonRecord.put(JavaBaseConstants.COLUMN_NAME_AB_LOADED_AT, (Long) null); + } else { + jsonRecord.put(JavaBaseConstants.COLUMN_NAME_AB_ID, id.toString()); + jsonRecord.put(JavaBaseConstants.COLUMN_NAME_EMITTED_AT, recordMessage.getEmittedAt()); + } jsonRecord.setAll((ObjectNode) recordMessage.getData()); return converter.convertToGenericDataRecord(WRITER.writeValueAsBytes(jsonRecord), schema); diff --git a/airbyte-integrations/bases/base-java-s3/src/test/java/io/airbyte/integrations/destination/s3/avro/AvroSerializedBufferTest.java b/airbyte-integrations/bases/base-java-s3/src/test/java/io/airbyte/integrations/destination/s3/avro/AvroSerializedBufferTest.java index 2bc57eedfcda..334fae868442 100644 --- a/airbyte-integrations/bases/base-java-s3/src/test/java/io/airbyte/integrations/destination/s3/avro/AvroSerializedBufferTest.java +++ b/airbyte-integrations/bases/base-java-s3/src/test/java/io/airbyte/integrations/destination/s3/avro/AvroSerializedBufferTest.java @@ -10,6 +10,7 @@ import com.fasterxml.jackson.databind.JsonNode; import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.base.DestinationConfig; import io.airbyte.integrations.destination.record_buffer.BufferStorage; import io.airbyte.integrations.destination.record_buffer.FileBuffer; import io.airbyte.integrations.destination.record_buffer.InMemoryBuffer; @@ -27,6 +28,7 @@ import org.apache.avro.file.SeekableByteArrayInput; import org.apache.avro.generic.GenericData.Record; import org.apache.avro.generic.GenericDatumReader; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; public class AvroSerializedBufferTest { @@ -49,6 +51,11 @@ public class AvroSerializedBufferTest { Field.of("nested_column", JsonSchemaType.OBJECT)); private static final ConfiguredAirbyteCatalog catalog = CatalogHelpers.createConfiguredAirbyteCatalog(STREAM, null, FIELDS); + @BeforeAll + public static void setup() { + DestinationConfig.initialize(Jsons.deserialize("{}")); + } + @Test public void testSnappyAvroWriter() throws Exception { final S3AvroFormatConfig config = new S3AvroFormatConfig(Jsons.jsonNode(Map.of("compression_codec", Map.of( diff --git a/airbyte-integrations/bases/base-java-s3/src/test/java/io/airbyte/integrations/destination/s3/parquet/ParquetSerializedBufferTest.java b/airbyte-integrations/bases/base-java-s3/src/test/java/io/airbyte/integrations/destination/s3/parquet/ParquetSerializedBufferTest.java index 2f2911cb51ac..a9ca83f66389 100644 --- a/airbyte-integrations/bases/base-java-s3/src/test/java/io/airbyte/integrations/destination/s3/parquet/ParquetSerializedBufferTest.java +++ b/airbyte-integrations/bases/base-java-s3/src/test/java/io/airbyte/integrations/destination/s3/parquet/ParquetSerializedBufferTest.java @@ -11,6 +11,7 @@ import com.amazonaws.util.IOUtils; import com.fasterxml.jackson.databind.JsonNode; import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.base.DestinationConfig; import io.airbyte.integrations.destination.record_buffer.SerializableBuffer; import io.airbyte.integrations.destination.s3.S3DestinationConfig; import io.airbyte.protocol.models.Field; @@ -33,6 +34,7 @@ import org.apache.hadoop.fs.Path; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.hadoop.ParquetReader; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; public class ParquetSerializedBufferTest { @@ -60,6 +62,11 @@ public class ParquetSerializedBufferTest { Field.of("datetime_with_timezone", JsonSchemaType.STRING_TIMESTAMP_WITH_TIMEZONE)); private static final ConfiguredAirbyteCatalog catalog = CatalogHelpers.createConfiguredAirbyteCatalog(STREAM, null, FIELDS); + @BeforeAll + public static void setup() { + DestinationConfig.initialize(Jsons.deserialize("{}")); + } + @Test public void testUncompressedParquetWriter() throws Exception { final S3DestinationConfig config = S3DestinationConfig.getS3DestinationConfig(Jsons.jsonNode(Map.of( diff --git a/airbyte-integrations/bases/base-java/run_with_normalization.sh b/airbyte-integrations/bases/base-java/run_with_normalization.sh index 4f59898e9268..f61cfea63b9a 100755 --- a/airbyte-integrations/bases/base-java/run_with_normalization.sh +++ b/airbyte-integrations/bases/base-java/run_with_normalization.sh @@ -6,10 +6,34 @@ set -o pipefail destination_exit_code=$? echo '{"type": "LOG","log":{"level":"INFO","message":"Destination process done (exit code '"$destination_exit_code"')"}}' +# store original args +args=$@ + +while [ $# -ne 0 ]; do + case "$1" in + --config) + CONFIG_FILE="$2" + shift 2 + ;; + *) + # move on + shift + ;; + esac +done + +# restore original args after shifts +set -- $args + +USE_1S1T_FORMAT="false" +if [[ -s "$CONFIG_FILE" ]]; then + USE_1S1T_FORMAT=$(jq -r '.use_1s1t_format' "$CONFIG_FILE") +fi + if test "$1" != 'write' then normalization_exit_code=0 -elif test "$NORMALIZATION_TECHNIQUE" = 'LEGACY' +elif test "$NORMALIZATION_TECHNIQUE" = 'LEGACY' && test "$USE_1S1T_FORMAT" != "true" then echo '{"type": "LOG","log":{"level":"INFO","message":"Starting in-connector normalization"}}' # the args in a write command are `write --catalog foo.json --config bar.json` diff --git a/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/DestinationConfig.java b/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/DestinationConfig.java new file mode 100644 index 000000000000..bd27cb2255f3 --- /dev/null +++ b/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/DestinationConfig.java @@ -0,0 +1,75 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.base; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.annotations.VisibleForTesting; +import jakarta.inject.Singleton; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Singleton of destination config for easy lookup of values. + */ +@Singleton +public class DestinationConfig { + + private static final Logger LOGGER = LoggerFactory.getLogger(DestinationConfig.class); + + private static DestinationConfig config; + + @VisibleForTesting + protected JsonNode root; + + private DestinationConfig() {} + + public static void initialize(final JsonNode root) { + if (config == null) { + if (root == null) { + throw new IllegalArgumentException("Cannot create DestinationConfig from null."); + } + config = new DestinationConfig(); + config.root = root; + } else { + LOGGER.warn("Singleton was already initialized."); + } + } + + public static DestinationConfig getInstance() { + if (config == null) { + throw new IllegalStateException("Singleton not initialized."); + } + return config; + } + + public JsonNode getNodeValue(final String key) { + final JsonNode node = config.root.get(key); + if (node == null) { + LOGGER.debug("Cannot find node with key {} ", key); + } + return node; + } + + // string value, otherwise empty string + public String getTextValue(final String key) { + final JsonNode node = getNodeValue(key); + if (node == null || !node.isTextual()) { + LOGGER.debug("Cannot retrieve text value for node with key {}", key); + return ""; + } + return node.asText(); + } + + // boolean value, otherwise false + public Boolean getBooleanValue(final String key) { + final JsonNode node = getNodeValue(key); + if (node == null || !node.isBoolean()) { + LOGGER.debug("Cannot retrieve boolean value for node with key {}", key); + return false; + } + return node.asBoolean(); + } + +} diff --git a/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/IntegrationRunner.java b/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/IntegrationRunner.java index 3fba41ef0f54..68bbedb3051f 100644 --- a/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/IntegrationRunner.java +++ b/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/IntegrationRunner.java @@ -144,6 +144,8 @@ private void runInternal(final IntegrationConfig parsed) throws Exception { case WRITE -> { final JsonNode config = parseConfig(parsed.getConfigPath()); validateConfig(integration.spec().getConnectionSpecification(), config, "WRITE"); + // save config to singleton + DestinationConfig.initialize(config); final ConfiguredAirbyteCatalog catalog = parseConfig(parsed.getCatalogPath(), ConfiguredAirbyteCatalog.class); final Procedure consumeWriteStreamCallable = () -> { diff --git a/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/JavaBaseConstants.java b/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/JavaBaseConstants.java index 4b3a4896dc4a..8d4bec36f3fa 100644 --- a/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/JavaBaseConstants.java +++ b/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/JavaBaseConstants.java @@ -20,4 +20,11 @@ private JavaBaseConstants() {} public static final String COLUMN_NAME_EMITTED_AT = "_airbyte_emitted_at"; public static final String COLUMN_NAME_DATA = "_airbyte_data"; + // destination v2 + public static final String COLUMN_NAME_AB_RAW_ID = "_airbyte_raw_id"; + public static final String COLUMN_NAME_AB_LOADED_AT = "_airbyte_loaded_at"; + public static final String COLUMN_NAME_AB_EXTRACTED_AT = "_airbyte_extracted_at"; + + public static final String AIRBYTE_NAMESPACE_SCHEMA = "airbyte"; + } diff --git a/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/TypingAndDedupingFlag.java b/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/TypingAndDedupingFlag.java new file mode 100644 index 000000000000..a3fff99346e1 --- /dev/null +++ b/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/TypingAndDedupingFlag.java @@ -0,0 +1,13 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.base; + +public class TypingAndDedupingFlag { + + public static final boolean isDestinationV2() { + return DestinationConfig.getInstance().getBooleanValue("use_1s1t_format"); + } + +} diff --git a/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/destination/buffered_stream_consumer/BufferedStreamConsumer.java b/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/destination/buffered_stream_consumer/BufferedStreamConsumer.java index efdbd2a019cd..b218bf6c29bc 100644 --- a/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/destination/buffered_stream_consumer/BufferedStreamConsumer.java +++ b/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/destination/buffered_stream_consumer/BufferedStreamConsumer.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.base.Strings; import io.airbyte.commons.functional.CheckedFunction; import io.airbyte.commons.json.Jsons; import io.airbyte.integrations.base.AirbyteMessageConsumer; @@ -92,7 +93,13 @@ public class BufferedStreamConsumer extends FailureTrackingAirbyteMessageConsume private Instant nextFlushDeadline; private final Duration bufferFlushFrequency; + private final String defaultNamespace; + /** + * Feel free to continue using this in non-1s1t destinations - it may be easier to use. However, + * 1s1t destinations should prefer the version which accepts a {@code defaultNamespace}. + */ + @Deprecated public BufferedStreamConsumer(final Consumer outputRecordCollector, final OnStartFunction onStart, final BufferingStrategy bufferingStrategy, @@ -105,7 +112,27 @@ public BufferedStreamConsumer(final Consumer outputRecordCollect onClose, catalog, isValidRecord, - Duration.ofMinutes(15)); + Duration.ofMinutes(15), + // This is purely for backwards compatibility. Many older destinations handle this internally. + // Starting with Destinations V2, we recommend passing in an explicit namespace. + null); + } + + public BufferedStreamConsumer(final Consumer outputRecordCollector, + final OnStartFunction onStart, + final BufferingStrategy bufferingStrategy, + final OnCloseFunction onClose, + final ConfiguredAirbyteCatalog catalog, + final CheckedFunction isValidRecord, + final String defaultNamespace) { + this(outputRecordCollector, + onStart, + bufferingStrategy, + onClose, + catalog, + isValidRecord, + Duration.ofMinutes(15), + defaultNamespace); } /* @@ -119,7 +146,8 @@ public BufferedStreamConsumer(final Consumer outputRecordCollect final OnCloseFunction onClose, final ConfiguredAirbyteCatalog catalog, final CheckedFunction isValidRecord, - final Duration flushFrequency) { + final Duration flushFrequency, + final String defaultNamespace) { this.outputRecordCollector = outputRecordCollector; this.hasStarted = false; this.hasClosed = false; @@ -132,6 +160,7 @@ public BufferedStreamConsumer(final Consumer outputRecordCollect this.bufferingStrategy = bufferingStrategy; this.stateManager = new DefaultDestStateLifecycleManager(); this.bufferFlushFrequency = flushFrequency; + this.defaultNamespace = defaultNamespace; } @Override @@ -157,7 +186,11 @@ protected void acceptTracked(final AirbyteMessage message) throws Exception { Preconditions.checkState(hasStarted, "Cannot accept records until consumer has started"); if (message.getType() == Type.RECORD) { final AirbyteRecordMessage record = message.getRecord(); - final AirbyteStreamNameNamespacePair stream = AirbyteStreamNameNamespacePair.fromRecordMessage(record); + if (Strings.isNullOrEmpty(record.getNamespace())) { + record.setNamespace(defaultNamespace); + } + final AirbyteStreamNameNamespacePair stream; + stream = AirbyteStreamNameNamespacePair.fromRecordMessage(record); // if stream is not part of list of streams to sync to then throw invalid stream exception if (!streamNames.contains(stream)) { diff --git a/airbyte-integrations/bases/base-java/src/test/java/io/airbyte/integrations/base/DestinationConfigTest.java b/airbyte-integrations/bases/base-java/src/test/java/io/airbyte/integrations/base/DestinationConfigTest.java new file mode 100644 index 000000000000..00182f989c26 --- /dev/null +++ b/airbyte-integrations/bases/base-java/src/test/java/io/airbyte/integrations/base/DestinationConfigTest.java @@ -0,0 +1,65 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.base; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import com.fasterxml.jackson.databind.JsonNode; +import io.airbyte.commons.json.Jsons; +import org.junit.jupiter.api.Test; + +public class DestinationConfigTest { + + private static final String JSON = """ + { + "foo": "bar", + "baz": true + } + """; + + private static final JsonNode NODE = Jsons.deserialize(JSON); + + @Test + public void testInitialization() { + // bad initialization + assertThrows(IllegalArgumentException.class, () -> DestinationConfig.initialize(null)); + assertThrows(IllegalStateException.class, DestinationConfig::getInstance); + + // good initialization + DestinationConfig.initialize(NODE); + assertNotNull(DestinationConfig.getInstance()); + assertEquals(NODE, DestinationConfig.getInstance().root); + + // initializing again doesn't change the config + final JsonNode nodeUnused = Jsons.deserialize("{}"); + DestinationConfig.initialize(nodeUnused); + assertEquals(NODE, DestinationConfig.getInstance().root); + } + + @Test + public void testValues() { + DestinationConfig.initialize(NODE); + + assertEquals("bar", DestinationConfig.getInstance().getTextValue("foo")); + assertEquals("", DestinationConfig.getInstance().getTextValue("baz")); + + assertFalse(DestinationConfig.getInstance().getBooleanValue("foo")); + assertTrue(DestinationConfig.getInstance().getBooleanValue("baz")); + + // non-existent key + assertEquals("", DestinationConfig.getInstance().getTextValue("blah")); + assertFalse(DestinationConfig.getInstance().getBooleanValue("blah")); + + assertEquals(Jsons.deserialize("\"bar\""), DestinationConfig.getInstance().getNodeValue("foo")); + assertEquals(Jsons.deserialize("true"), DestinationConfig.getInstance().getNodeValue("baz")); + assertNull(DestinationConfig.getInstance().getNodeValue("blah")); + } + +} diff --git a/airbyte-integrations/bases/base-java/src/test/java/io/airbyte/integrations/destination/buffered_stream_consumer/BufferedStreamConsumerTest.java b/airbyte-integrations/bases/base-java/src/test/java/io/airbyte/integrations/destination/buffered_stream_consumer/BufferedStreamConsumerTest.java index 11ef6402e14e..d6491eef6058 100644 --- a/airbyte-integrations/bases/base-java/src/test/java/io/airbyte/integrations/destination/buffered_stream_consumer/BufferedStreamConsumerTest.java +++ b/airbyte-integrations/bases/base-java/src/test/java/io/airbyte/integrations/destination/buffered_stream_consumer/BufferedStreamConsumerTest.java @@ -358,7 +358,8 @@ private BufferedStreamConsumer getConsumerWithFlushFrequency() { onClose, CATALOG, isValidRecord, - Duration.ofSeconds(PERIODIC_BUFFER_FREQUENCY)); + Duration.ofSeconds(PERIODIC_BUFFER_FREQUENCY), + null); return flushFrequencyConsumer; } diff --git a/airbyte-integrations/bases/base-normalization/build.gradle b/airbyte-integrations/bases/base-normalization/build.gradle index d1d2d4c7c7b0..03740e0f9019 100644 --- a/airbyte-integrations/bases/base-normalization/build.gradle +++ b/airbyte-integrations/bases/base-normalization/build.gradle @@ -58,11 +58,11 @@ def buildAirbyteDocker(String customConnector) { arch = 'linux/arm64' } - def baseCommand = ['docker', 'buildx', 'build', '--load', '--platform', arch, '-f', getDockerfile(customConnector), '-t', getImageNameWithTag(customConnector), '.'] - // println("Building normalization container: " + baseCommand.join(" ")) + def cmdArray = ['docker', 'buildx', 'build', '--load', '--platform', arch, '-f', getDockerfile(customConnector), '-t', getImageNameWithTag(customConnector), '.'] + // println("Building normalization container: " + cmdArray.join(" ")) return { - commandLine baseCommand + commandLine cmdArray } } diff --git a/airbyte-integrations/bases/base-normalization/dbt.Dockerfile b/airbyte-integrations/bases/base-normalization/dbt.Dockerfile new file mode 100644 index 000000000000..09b0e3c94064 --- /dev/null +++ b/airbyte-integrations/bases/base-normalization/dbt.Dockerfile @@ -0,0 +1,3 @@ +# This dockerfile only exists to pull and re-export this image converted to the local arch of this machine +# It is then consumed by the Dockerfile in this direcotry as "fishtownanalytics/dbt:1.0.0-dev" +FROM fishtownanalytics/dbt:1.0.0 \ No newline at end of file diff --git a/airbyte-integrations/bases/base-typing-deduping-test/build.gradle b/airbyte-integrations/bases/base-typing-deduping-test/build.gradle new file mode 100644 index 000000000000..5c786c2f79c0 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/build.gradle @@ -0,0 +1,15 @@ +plugins { + id 'java-library' +} + +dependencies { + implementation project(':airbyte-config-oss:config-models-oss') + implementation project(':airbyte-connector-test-harnesses:acceptance-test-harness') + implementation project(':airbyte-integrations:bases:base-typing-deduping') + implementation libs.airbyte.protocol + + implementation(enforcedPlatform('org.junit:junit-bom:5.8.2')) + implementation 'org.junit.jupiter:junit-jupiter-api' + implementation 'org.junit.jupiter:junit-jupiter-params' + implementation 'org.mockito:mockito-core:4.6.1' +} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/BaseTypingDedupingTest.java b/airbyte-integrations/bases/base-typing-deduping-test/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/BaseTypingDedupingTest.java new file mode 100644 index 000000000000..85ca77f81b69 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/BaseTypingDedupingTest.java @@ -0,0 +1,573 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.base.destination.typing_deduping; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableMap; +import io.airbyte.commons.features.EnvVariableFeatureFlags; +import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.lang.Exceptions; +import io.airbyte.commons.resources.MoreResources; +import io.airbyte.configoss.WorkerDestinationConfig; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.AirbyteProtocolType; +import io.airbyte.protocol.models.v0.AirbyteMessage; +import io.airbyte.protocol.models.v0.AirbyteStream; +import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair; +import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog; +import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream; +import io.airbyte.protocol.models.v0.DestinationSyncMode; +import io.airbyte.protocol.models.v0.SyncMode; +import io.airbyte.workers.internal.AirbyteDestination; +import io.airbyte.workers.internal.DefaultAirbyteDestination; +import io.airbyte.workers.process.AirbyteIntegrationLauncher; +import io.airbyte.workers.process.DockerProcessFactory; +import io.airbyte.workers.process.ProcessFactory; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.stream.Stream; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is loosely based on standard-destination-tests's DestinationAcceptanceTest class. The + * sync-running code is copy-pasted from there. + *

+ * All tests use a single stream, whose schema is defined in {@code resources/schema.json}. Each + * test case constructs a ConfiguredAirbyteCatalog dynamically. + *

+ * For sync modes which use a primary key, the stream provides a composite key of (id1, id2). For + * sync modes which use a cursor, the stream provides an updated_at field. The stream also has an + * _ab_cdc_deleted_at field. + */ +// If you're running from inside intellij, you must run your specific subclass to get concurrent +// execution. +@Execution(ExecutionMode.CONCURRENT) +public abstract class BaseTypingDedupingTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(BaseTypingDedupingTest.class); + private static final JsonNode SCHEMA; + static { + try { + SCHEMA = Jsons.deserialize(MoreResources.readResource("schema.json")); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + private static final RecordDiffer DIFFER = new RecordDiffer( + Pair.of("id1", AirbyteProtocolType.INTEGER), + Pair.of("id2", AirbyteProtocolType.INTEGER), + Pair.of("updated_at", AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE)); + + private String randomSuffix; + private JsonNode config; + private String streamNamespace; + private String streamName; + private List streamsToTearDown; + + /** + * @return the docker image to run, e.g. {@code "airbyte/destination-bigquery:dev"}. + */ + protected abstract String getImageName(); + + /** + * Get the destination connector config. Subclasses may use this method for other setup work, e.g. + * opening a connection to the destination. + *

+ * Subclasses should _not_ start testcontainers in this method; that belongs in a BeforeAll method. + * The tests in this class are intended to be run concurrently on a shared database and will not + * interfere with each other. + *

+ * Sublcasses which need access to the config may use {@link #getConfig()}. + */ + protected abstract JsonNode generateConfig() throws Exception; + + /** + * For a given stream, return the records that exist in the destination's raw table. Each record + * must be in the format {"_airbyte_raw_id": "...", "_airbyte_extracted_at": "...", + * "_airbyte_loaded_at": "...", "_airbyte_data": {fields...}}. + *

+ * The {@code _airbyte_data} column must be an + * {@link com.fasterxml.jackson.databind.node.ObjectNode} (i.e. it cannot be a string value). + *

+ * streamNamespace may be null, in which case you should query from the default namespace. + */ + protected abstract List dumpRawTableRecords(String streamNamespace, String streamName) throws Exception; + + /** + * For a given stream, return the records that exist in the destination's final table. Each record + * must be in the format {"_airbyte_raw_id": "...", "_airbyte_extracted_at": "...", "_airbyte_meta": + * {...}, "field1": ..., "field2": ..., ...}. + *

+ * For JSON-valued columns, there is some nuance: a SQL null should be represented as a missing + * entry, whereas a JSON null should be represented as a + * {@link com.fasterxml.jackson.databind.node.NullNode}. For example, in the JSON blob {"name": + * null}, the `name` field is a JSON null, and the `address` field is a SQL null. + *

+ * The corresponding SQL looks like + * {@code INSERT INTO ... (name, address) VALUES ('null' :: jsonb, NULL)}. + *

+ * streamNamespace may be null, in which case you should query from the default namespace. + */ + protected abstract List dumpFinalTableRecords(String streamNamespace, String streamName) throws Exception; + + /** + * Delete any resources in the destination associated with this stream AND its namespace. We need + * this because we write raw tables to a shared {@code airbyte} namespace, which we can't drop + * wholesale. Must handle the case where the table/namespace doesn't exist (e.g. if the connector + * crashed without writing any data). + *

+ * In general, this should resemble + * {@code DROP TABLE IF EXISTS airbyte._; DROP SCHEMA IF EXISTS }. + */ + protected abstract void teardownStreamAndNamespace(String streamNamespace, String streamName) throws Exception; + + /** + * @return A suffix which is different for each concurrent test run. + */ + protected synchronized String getUniqueSuffix() { + if (randomSuffix == null) { + randomSuffix = "_" + RandomStringUtils.randomAlphabetic(5).toLowerCase(); + } + return randomSuffix; + } + + protected JsonNode getConfig() { + return config; + } + + @BeforeEach + public void setup() throws Exception { + config = generateConfig(); + streamNamespace = "typing_deduping_test" + getUniqueSuffix(); + streamName = "test_stream" + getUniqueSuffix(); + streamsToTearDown = new ArrayList<>(); + LOGGER.info("Using stream namespace {} and name {}", streamNamespace, streamName); + } + + @AfterEach + public void teardown() throws Exception { + for (AirbyteStreamNameNamespacePair streamId : streamsToTearDown) { + teardownStreamAndNamespace(streamId.getNamespace(), streamId.getName()); + } + } + + /** + * Starting with an empty destination, execute a full refresh overwrite sync. Verify that the + * records are written to the destination table. Then run a second sync, and verify that the records + * are overwritten. + */ + @Test + public void fullRefreshOverwrite() throws Exception { + ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(List.of( + new ConfiguredAirbyteStream() + .withSyncMode(SyncMode.FULL_REFRESH) + .withDestinationSyncMode(DestinationSyncMode.OVERWRITE) + .withStream(new AirbyteStream() + .withNamespace(streamNamespace) + .withName(streamName) + .withJsonSchema(SCHEMA)))); + + // First sync + List messages1 = readMessages("sync1_messages.jsonl"); + + runSync(catalog, messages1); + + List expectedRawRecords1 = readRecords("sync1_expectedrecords_nondedup_raw.jsonl"); + List expectedFinalRecords1 = readRecords("sync1_expectedrecords_nondedup_final.jsonl"); + verifySyncResult(expectedRawRecords1, expectedFinalRecords1); + + // Second sync + List messages2 = readMessages("sync2_messages.jsonl"); + + runSync(catalog, messages2); + + List expectedRawRecords2 = readRecords("sync2_expectedrecords_fullrefresh_overwrite_raw.jsonl"); + List expectedFinalRecords2 = readRecords("sync2_expectedrecords_fullrefresh_overwrite_final.jsonl"); + verifySyncResult(expectedRawRecords2, expectedFinalRecords2); + } + + /** + * Starting with an empty destination, execute a full refresh append sync. Verify that the records + * are written to the destination table. Then run a second sync, and verify that the old and new + * records are all present. + */ + @Test + public void fullRefreshAppend() throws Exception { + ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(List.of( + new ConfiguredAirbyteStream() + .withSyncMode(SyncMode.FULL_REFRESH) + .withDestinationSyncMode(DestinationSyncMode.APPEND) + .withStream(new AirbyteStream() + .withNamespace(streamNamespace) + .withName(streamName) + .withJsonSchema(SCHEMA)))); + + // First sync + List messages1 = readMessages("sync1_messages.jsonl"); + + runSync(catalog, messages1); + + List expectedRawRecords1 = readRecords("sync1_expectedrecords_nondedup_raw.jsonl"); + List expectedFinalRecords1 = readRecords("sync1_expectedrecords_nondedup_final.jsonl"); + verifySyncResult(expectedRawRecords1, expectedFinalRecords1); + + // Second sync + List messages2 = readMessages("sync2_messages.jsonl"); + + runSync(catalog, messages2); + + List expectedRawRecords2 = readRecords("sync2_expectedrecords_fullrefresh_append_raw.jsonl"); + List expectedFinalRecords2 = readRecords("sync2_expectedrecords_fullrefresh_append_final.jsonl"); + verifySyncResult(expectedRawRecords2, expectedFinalRecords2); + } + + /** + * Starting with an empty destination, execute an incremental append sync. + *

+ * This is (not so secretly) identical to {@link #fullRefreshAppend()}, and uses the same set of + * expected records. Incremental as a concept only exists in the source. From the destination's + * perspective, we only care about the destination sync mode. + */ + @Test + public void incrementalAppend() throws Exception { + ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(List.of( + new ConfiguredAirbyteStream() + // These two lines are literally the only difference between this test and fullRefreshAppend + .withSyncMode(SyncMode.INCREMENTAL) + .withCursorField(List.of("updated_at")) + .withDestinationSyncMode(DestinationSyncMode.APPEND) + .withStream(new AirbyteStream() + .withNamespace(streamNamespace) + .withName(streamName) + .withJsonSchema(SCHEMA)))); + + // First sync + List messages1 = readMessages("sync1_messages.jsonl"); + + runSync(catalog, messages1); + + List expectedRawRecords1 = readRecords("sync1_expectedrecords_nondedup_raw.jsonl"); + List expectedFinalRecords1 = readRecords("sync1_expectedrecords_nondedup_final.jsonl"); + verifySyncResult(expectedRawRecords1, expectedFinalRecords1); + + // Second sync + List messages2 = readMessages("sync2_messages.jsonl"); + + runSync(catalog, messages2); + + List expectedRawRecords2 = readRecords("sync2_expectedrecords_fullrefresh_append_raw.jsonl"); + List expectedFinalRecords2 = readRecords("sync2_expectedrecords_fullrefresh_append_final.jsonl"); + verifySyncResult(expectedRawRecords2, expectedFinalRecords2); + } + + /** + * Starting with an empty destination, execute an incremental dedup sync. Verify that the records + * are written to the destination table. Then run a second sync, and verify that the raw/final + * tables contain the correct records. + */ + @Test + public void incrementalDedup() throws Exception { + ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(List.of( + new ConfiguredAirbyteStream() + .withSyncMode(SyncMode.INCREMENTAL) + .withCursorField(List.of("updated_at")) + .withDestinationSyncMode(DestinationSyncMode.APPEND_DEDUP) + .withPrimaryKey(List.of(List.of("id1"), List.of("id2"))) + .withStream(new AirbyteStream() + .withNamespace(streamNamespace) + .withName(streamName) + .withJsonSchema(SCHEMA)))); + + // First sync + List messages1 = readMessages("sync1_messages.jsonl"); + + runSync(catalog, messages1); + + List expectedRawRecords1 = readRecords("sync1_expectedrecords_dedup_raw.jsonl"); + List expectedFinalRecords1 = readRecords("sync1_expectedrecords_dedup_final.jsonl"); + verifySyncResult(expectedRawRecords1, expectedFinalRecords1); + + // Second sync + List messages2 = readMessages("sync2_messages.jsonl"); + + runSync(catalog, messages2); + + List expectedRawRecords2 = readRecords("sync2_expectedrecords_incremental_dedup_raw.jsonl"); + List expectedFinalRecords2 = readRecords("sync2_expectedrecords_incremental_dedup_final.jsonl"); + verifySyncResult(expectedRawRecords2, expectedFinalRecords2); + } + + /** + * Identical to {@link #incrementalDedup()}, except that the stream has no namespace. + */ + @Test + public void incrementalDedupDefaultNamespace() throws Exception { + ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(List.of( + new ConfiguredAirbyteStream() + .withSyncMode(SyncMode.INCREMENTAL) + .withCursorField(List.of("updated_at")) + .withDestinationSyncMode(DestinationSyncMode.APPEND_DEDUP) + .withPrimaryKey(List.of(List.of("id1"), List.of("id2"))) + .withStream(new AirbyteStream() + // NB: we don't call `withNamespace` here + .withName(streamName) + .withJsonSchema(SCHEMA)))); + + // First sync + List messages1 = readMessages("sync1_messages.jsonl", null, streamName); + + runSync(catalog, messages1); + + List expectedRawRecords1 = readRecords("sync1_expectedrecords_dedup_raw.jsonl"); + List expectedFinalRecords1 = readRecords("sync1_expectedrecords_dedup_final.jsonl"); + verifySyncResult(expectedRawRecords1, expectedFinalRecords1, null, streamName); + + // Second sync + List messages2 = readMessages("sync2_messages.jsonl", null, streamName); + + runSync(catalog, messages2); + + List expectedRawRecords2 = readRecords("sync2_expectedrecords_incremental_dedup_raw.jsonl"); + List expectedFinalRecords2 = readRecords("sync2_expectedrecords_incremental_dedup_final.jsonl"); + verifySyncResult(expectedRawRecords2, expectedFinalRecords2, null, streamName); + } + + @Test + @Disabled("Not yet implemented") + public void testLineBreakCharacters() throws Exception { + // TODO verify that we can handle strings with interesting characters + // build an airbyterecordmessage using something like this, and add it to the input messages: + Jsons.jsonNode(ImmutableMap.builder() + .put("id", 1) + .put("currency", "USD\u2028") + .put("date", "2020-03-\n31T00:00:00Z\r") + // TODO(sherifnada) hack: write decimals with sigfigs because Snowflake stores 10.1 as "10" which + // fails destination tests + .put("HKD", 10.1) + .put("NZD", 700.1) + .build()); + } + + @Test + @Disabled("Not yet implemented") + public void testIncrementalSyncDropOneColumn() throws Exception { + // TODO in incremental dedup mode: run a sync, remove a column from the schema, run another sync + // verify that the column is dropped from the destination table + } + + @Test + @Disabled("Not yet implemented") + public void testSyncUsesAirbyteStreamNamespaceIfNotNull() throws Exception { + // TODO duplicate this test for each sync mode. Run 1st+2nd syncs using a stream with null + // namespace: + ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(List.of( + new ConfiguredAirbyteStream() + .withSyncMode(SyncMode.FULL_REFRESH) + .withCursorField(List.of("updated_at")) + .withDestinationSyncMode(DestinationSyncMode.OVERWRITE) + .withPrimaryKey(List.of(List.of("id1"), List.of("id2"))) + .withStream(new AirbyteStream() + .withNamespace(null) + .withName(streamName) + .withJsonSchema(SCHEMA)))); + } + + // TODO duplicate this test for each sync mode. Run 1st+2nd syncs using two streams with the same + // name but different namespace + // TODO maybe we don't even need the single-stream versions... + /** + * Identical to {@link #incrementalDedup()}, except there are two streams with the same name and + * different namespace. + */ + @Test + public void incrementalDedupIdenticalName() throws Exception { + String namespace1 = streamNamespace + "_1"; + String namespace2 = streamNamespace + "_2"; + ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(List.of( + new ConfiguredAirbyteStream() + .withSyncMode(SyncMode.INCREMENTAL) + .withCursorField(List.of("updated_at")) + .withDestinationSyncMode(DestinationSyncMode.APPEND_DEDUP) + .withPrimaryKey(List.of(List.of("id1"), List.of("id2"))) + .withStream(new AirbyteStream() + .withNamespace(namespace1) + .withName(streamName) + .withJsonSchema(SCHEMA)), + new ConfiguredAirbyteStream() + .withSyncMode(SyncMode.INCREMENTAL) + .withCursorField(List.of("updated_at")) + .withDestinationSyncMode(DestinationSyncMode.APPEND_DEDUP) + .withPrimaryKey(List.of(List.of("id1"), List.of("id2"))) + .withStream(new AirbyteStream() + .withNamespace(namespace2) + .withName(streamName) + .withJsonSchema(SCHEMA)))); + + // First sync + // Read the same set of messages for both streams + List messages1 = Stream.concat( + readMessages("sync1_messages.jsonl", namespace1, streamName).stream(), + readMessages("sync1_messages.jsonl", namespace2, streamName).stream()).toList(); + + runSync(catalog, messages1); + + List expectedRawRecords1 = readRecords("sync1_expectedrecords_dedup_raw.jsonl"); + List expectedFinalRecords1 = readRecords("sync1_expectedrecords_dedup_final.jsonl"); + verifySyncResult(expectedRawRecords1, expectedFinalRecords1, namespace1, streamName); + verifySyncResult(expectedRawRecords1, expectedFinalRecords1, namespace2, streamName); + + // Second sync + List messages2 = Stream.concat( + readMessages("sync2_messages.jsonl", namespace1, streamName).stream(), + readMessages("sync2_messages.jsonl", namespace2, streamName).stream()).toList(); + + runSync(catalog, messages2); + + List expectedRawRecords2 = readRecords("sync2_expectedrecords_incremental_dedup_raw.jsonl"); + List expectedFinalRecords2 = readRecords("sync2_expectedrecords_incremental_dedup_final.jsonl"); + verifySyncResult(expectedRawRecords2, expectedFinalRecords2, namespace1, streamName); + verifySyncResult(expectedRawRecords2, expectedFinalRecords2, namespace2, streamName); + } + + @Test + @Disabled("Not yet implemented") + public void testSyncNotFailsWithNewFields() throws Exception { + // TODO duplicate this test for each sync mode. Run a sync, then add a new field to the schema, then + // run another sync + // We might want to write a test that verifies more general schema evolution (e.g. all valid + // evolutions) + } + + @Test + @Disabled("Not yet implemented") + public void testSyncWithLargeRecordBatch() throws Exception { + // TODO duplicate this test for each sync mode. Run a single sync with many records + /* + * copied from DATs: This serves to test MSSQL 2100 limit parameters in a single query. this means + * that for Airbyte insert data need to limit to ~ 700 records (3 columns for the raw tables) = 2100 + * params + * + * this maybe needs configuration per destination to specify that limit? + */ + } + + @Test + @Disabled("Not yet implemented") + public void testDataTypes() throws Exception { + // TODO duplicate this test for each sync mode. See DataTypeTestArgumentProvider for what this test + // does in DAT-land + // we probably don't want to do the exact same thing, but the general spirit of testing a wide range + // of values for every data type is approximately correct + // this test probably needs some configuration per destination to specify what values are supported? + } + + private void verifySyncResult(List expectedRawRecords, List expectedFinalRecords) throws Exception { + verifySyncResult(expectedRawRecords, expectedFinalRecords, streamNamespace, streamName); + } + + private void verifySyncResult(List expectedRawRecords, + List expectedFinalRecords, + String streamNamespace, + String streamName) + throws Exception { + List actualRawRecords = dumpRawTableRecords(streamNamespace, streamName); + List actualFinalRecords = dumpFinalTableRecords(streamNamespace, streamName); + DIFFER.verifySyncResult(expectedRawRecords, actualRawRecords, expectedFinalRecords, actualFinalRecords); + } + + private static List readRecords(String filename) throws IOException { + return MoreResources.readResource(filename).lines() + .map(String::trim) + .filter(line -> !line.isEmpty()) + .filter(line -> !line.startsWith("//")) + .map(Jsons::deserialize) + .toList(); + } + + private List readMessages(String filename) throws IOException { + return readMessages(filename, streamNamespace, streamName); + } + + private static List readMessages(String filename, String streamNamespace, String streamName) throws IOException { + return readRecords(filename).stream() + .map(record -> Jsons.convertValue(record, AirbyteMessage.class)) + .peek(message -> { + message.getRecord().setNamespace(streamNamespace); + message.getRecord().setStream(streamName); + }).toList(); + } + + /* + * !!!!!! WARNING !!!!!! The code below was mostly copypasted from DestinationAcceptanceTest. If you + * make edits here, you probably want to also edit there. + */ + + // These contain some state, so they are instanced per test (i.e. cannot be static) + private Path jobRoot; + private ProcessFactory processFactory; + + @BeforeEach + public void setupProcessFactory() throws IOException { + final Path testDir = Path.of("/tmp/airbyte_tests/"); + Files.createDirectories(testDir); + final Path workspaceRoot = Files.createTempDirectory(testDir, "test"); + jobRoot = Files.createDirectories(Path.of(workspaceRoot.toString(), "job")); + Path localRoot = Files.createTempDirectory(testDir, "output"); + processFactory = new DockerProcessFactory( + workspaceRoot, + workspaceRoot.toString(), + localRoot.toString(), + "host", + Collections.emptyMap()); + } + + private void runSync(ConfiguredAirbyteCatalog catalog, List messages) throws Exception { + catalog.getStreams().forEach(s -> streamsToTearDown.add(AirbyteStreamNameNamespacePair.fromAirbyteStream(s.getStream()))); + + final WorkerDestinationConfig destinationConfig = new WorkerDestinationConfig() + .withConnectionId(UUID.randomUUID()) + .withCatalog(convertProtocolObject(catalog, io.airbyte.protocol.models.ConfiguredAirbyteCatalog.class)) + .withDestinationConnectionConfiguration(config); + + final AirbyteDestination destination = new DefaultAirbyteDestination(new AirbyteIntegrationLauncher( + "0", + 0, + getImageName(), + processFactory, + null, + null, + false, + new EnvVariableFeatureFlags())); + + destination.start(destinationConfig, jobRoot, Collections.emptyMap()); + messages.forEach( + message -> Exceptions.toRuntime(() -> destination.accept(convertProtocolObject(message, io.airbyte.protocol.models.AirbyteMessage.class)))); + destination.notifyEndOfInput(); + + while (!destination.isFinished()) { + destination.attemptRead(); + } + + destination.close(); + } + + private static V0 convertProtocolObject(final V1 v1, final Class klass) { + return Jsons.object(Jsons.jsonNode(v1), klass); + } + +} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/RecordDiffer.java b/airbyte-integrations/bases/base-typing-deduping-test/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/RecordDiffer.java new file mode 100644 index 000000000000..846fb4a88bff --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/RecordDiffer.java @@ -0,0 +1,393 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.base.destination.typing_deduping; + +import static java.util.stream.Collectors.toList; +import static org.junit.jupiter.api.Assertions.assertAll; +import static org.junit.jupiter.api.Assertions.fail; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.collect.Streams; +import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.AirbyteProtocolType; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetTime; +import java.time.ZoneOffset; +import java.util.Arrays; +import java.util.Comparator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.Pair; + +/** + * Utility class to generate human-readable diffs between expected and actual records. Assumes 1s1t + * output format. + */ +public class RecordDiffer { + + private final Comparator recordIdentityComparator; + private final Comparator recordSortComparator; + private final Function recordIdentityExtractor; + + /** + * @param identifyingColumns Which fields constitute a unique record (typically PK+cursor). Do _not_ + * include extracted_at; it is handled automatically. + */ + @SafeVarargs + public RecordDiffer(final Pair... identifyingColumns) { + this.recordIdentityComparator = buildIdentityComparator(identifyingColumns); + this.recordSortComparator = recordIdentityComparator.thenComparing(record -> asString(record.get("_airbyte_raw_id"))); + this.recordIdentityExtractor = buildIdentityExtractor(identifyingColumns); + } + + /** + * In the expected records, a SQL null is represented as a JsonNode without that field at all, and a + * JSON null is represented as a NullNode. For example, in the JSON blob {"name": null}, the `name` + * field is a JSON null, and the `address` field is a SQL null. + */ + public void verifySyncResult(List expectedRawRecords, + List actualRawRecords, + List expectedFinalRecords, + List actualFinalRecords) { + assertAll( + () -> diffRawTableRecords(expectedRawRecords, actualRawRecords), + () -> diffFinalTableRecords(expectedFinalRecords, actualFinalRecords)); + } + + public void diffRawTableRecords(List expectedRecords, List actualRecords) { + String diff = diffRecords( + expectedRecords.stream().map(RecordDiffer::copyWithLiftedData).collect(toList()), + actualRecords.stream().map(RecordDiffer::copyWithLiftedData).collect(toList()), + recordIdentityComparator, + recordSortComparator, + recordIdentityExtractor); + + if (!diff.isEmpty()) { + fail("Raw table was incorrect.\n" + diff); + } + } + + public void diffFinalTableRecords(List expectedRecords, List actualRecords) { + String diff = diffRecords( + expectedRecords, + actualRecords, + recordIdentityComparator, + recordSortComparator, + recordIdentityExtractor); + + if (!diff.isEmpty()) { + fail("Final table was incorrect.\n" + diff); + } + } + + /** + * @return A copy of the record, but with all fields in _airbyte_data lifted to the top level. + */ + private static JsonNode copyWithLiftedData(JsonNode record) { + ObjectNode copy = record.deepCopy(); + copy.remove("_airbyte_data"); + Streams.stream(record.get("_airbyte_data").fields()).forEach(field -> { + if (!copy.has(field.getKey())) { + copy.set(field.getKey(), field.getValue()); + } else { + // This would only happen if the record has one of the metadata columns (e.g. _airbyte_raw_id) + // We don't support that in production, so we don't support it here either. + throw new RuntimeException("Cannot lift field " + field.getKey() + " because it already exists in the record."); + } + }); + return copy; + } + + /** + * Build a Comparator to detect equality between two records. It first compares all the identifying + * columns in order, and breaks ties using extracted_at. + */ + private Comparator buildIdentityComparator(Pair[] identifyingColumns) { + // Start with a noop comparator for convenience + Comparator comp = Comparator.comparing(record -> 0); + for (Pair column : identifyingColumns) { + comp = comp.thenComparing(record -> extract(record, column.getKey(), column.getValue())); + } + comp = comp.thenComparing(record -> asTimestampWithTimezone(record.get("_airbyte_extracted_at"))); + return comp; + } + + /** + * See {@link #buildIdentityComparator(Pair[])} for an explanation of dataExtractor. + */ + private Function buildIdentityExtractor(Pair[] identifyingColumns) { + return record -> Arrays.stream(identifyingColumns) + .map(column -> getPrintableFieldIfPresent(record, column.getKey())) + .collect(Collectors.joining(", ")) + + getPrintableFieldIfPresent(record, "_airbyte_extracted_at"); + } + + private static String getPrintableFieldIfPresent(JsonNode record, String field) { + if (record.has(field)) { + return field + "=" + record.get(field); + } else { + return ""; + } + } + + /** + * Generate a human-readable diff between the two lists. Assumes (in general) that two records with + * the same PK, cursor, and extracted_at are the same record. + *

+ * Verifies that all values specified in the expected records are correct (_including_ raw_id), and + * that no other fields are present (except for loaded_at and raw_id). We assume that it's + * impossible to verify loaded_at, since it's generated dynamically; however, we do provide the + * ability to assert on the exact raw_id if desired; we simply assume that raw_id is always expected + * to be present. + * + * @param identityComparator Returns 0 iff two records are the "same" record (i.e. have the same + * PK+cursor+extracted_at) + * @param sortComparator Behaves identically to identityComparator, but if two records are the same, + * breaks that tie using _airbyte_raw_id + * @param recordIdExtractor Dump the record's PK+cursor+extracted_at into a human-readable string + * @return The diff, or empty string if there were no differences + */ + private static String diffRecords(List originalExpectedRecords, + List originalActualRecords, + Comparator identityComparator, + Comparator sortComparator, + Function recordIdExtractor) { + List expectedRecords = originalExpectedRecords.stream().sorted(sortComparator).toList(); + List actualRecords = originalActualRecords.stream().sorted(sortComparator).toList(); + + // Iterate through both lists in parallel and compare each record. + // Build up an error message listing any incorrect, missing, or unexpected records. + String message = ""; + int expectedRecordIndex = 0; + int actualRecordIndex = 0; + while (expectedRecordIndex < expectedRecords.size() && actualRecordIndex < actualRecords.size()) { + JsonNode expectedRecord = expectedRecords.get(expectedRecordIndex); + JsonNode actualRecord = actualRecords.get(actualRecordIndex); + int compare = identityComparator.compare(expectedRecord, actualRecord); + if (compare == 0) { + // These records should be the same. Find the specific fields that are different and move on + // to the next records in both lists. + message += diffSingleRecord(recordIdExtractor, expectedRecord, actualRecord); + expectedRecordIndex++; + actualRecordIndex++; + } else if (compare < 0) { + // The expected record is missing from the actual records. Print it and move on to the next expected + // record. + message += "Row was expected but missing: " + expectedRecord + "\n"; + expectedRecordIndex++; + } else { + // There's an actual record which isn't present in the expected records. Print it and move on to the + // next actual record. + message += "Row was not expected but present: " + actualRecord + "\n"; + actualRecordIndex++; + } + } + // Tail loops in case we reached the end of one list before the other. + while (expectedRecordIndex < expectedRecords.size()) { + message += "Row was expected but missing: " + expectedRecords.get(expectedRecordIndex) + "\n"; + expectedRecordIndex++; + } + while (actualRecordIndex < actualRecords.size()) { + message += "Row was not expected but present: " + actualRecords.get(actualRecordIndex) + "\n"; + actualRecordIndex++; + } + + return message; + } + + private static String diffSingleRecord(Function recordIdExtractor, JsonNode expectedRecord, JsonNode actualRecord) { + boolean foundMismatch = false; + String mismatchedRecordMessage = "Row had incorrect data: " + recordIdExtractor.apply(expectedRecord) + "\n"; + // Iterate through each column in the expected record and compare it to the actual record's value. + for (String column : Streams.stream(expectedRecord.fieldNames()).sorted().toList()) { + // For all other columns, we can just compare their values directly. + JsonNode expectedValue = expectedRecord.get(column); + JsonNode actualValue = actualRecord.get(column); + if (!areJsonNodesEquivalent(expectedValue, actualValue)) { + mismatchedRecordMessage += generateFieldError("column " + column, expectedValue, actualValue); + foundMismatch = true; + } + } + // Then check the entire actual record for any columns that we weren't expecting. + LinkedHashMap extraColumns = checkForExtraOrNonNullFields(expectedRecord, actualRecord); + if (extraColumns.size() > 0) { + for (Map.Entry extraColumn : extraColumns.entrySet()) { + mismatchedRecordMessage += generateFieldError("column " + extraColumn.getKey(), null, extraColumn.getValue()); + foundMismatch = true; + } + } + if (foundMismatch) { + return mismatchedRecordMessage; + } else { + return ""; + } + } + + private static boolean areJsonNodesEquivalent(JsonNode expectedValue, JsonNode actualValue) { + if (expectedValue == null || actualValue == null) { + // If one of the values is null, then we expect both of them to be null. + return expectedValue == null && actualValue == null; + } else { + // Otherwise, we need to compare the actual values. + // This is kind of sketchy, but seems to work fine for the data we have in our test cases. + return expectedValue.equals(actualValue) + // equals() expects the two values to be the same class. + // We need to handle comparisons between e.g. LongNode and IntNode. + || (expectedValue.isIntegralNumber() && actualValue.isIntegralNumber() && expectedValue.asLong() == actualValue.asLong()) + || (expectedValue.isNumber() && actualValue.isNumber() && expectedValue.asDouble() == actualValue.asDouble()); + } + } + + /** + * Verify that all fields in the actual record are present in the expected record. This is primarily + * relevant for detecting fields that we expected to be null, but actually were not. See + * {@link BaseTypingDedupingTest#dumpFinalTableRecords(String, String)} for an explanation of how + * SQL/JSON nulls are represented in the expected record. + *

+ * This has the side benefit of detecting completely unexpected columns, which would be a very weird + * bug but is probably still useful to catch. + */ + private static LinkedHashMap checkForExtraOrNonNullFields(JsonNode expectedRecord, JsonNode actualRecord) { + LinkedHashMap extraFields = new LinkedHashMap<>(); + for (String column : Streams.stream(actualRecord.fieldNames()).sorted().toList()) { + // loaded_at and raw_id are generated dynamically, so we just ignore them. + if (!"_airbyte_loaded_at".equals(column) && !"_airbyte_raw_id".equals(column) && !expectedRecord.has(column)) { + extraFields.put(column, actualRecord.get(column)); + } + } + return extraFields; + } + + /** + * Produce a pretty-printed error message, e.g. " For column foo, expected 1 but got 2". The leading + * spaces are intentional, to make the message easier to read when it's embedded in a larger + * stacktrace. + */ + private static String generateFieldError(String fieldname, JsonNode expectedValue, JsonNode actualValue) { + String expectedString = expectedValue == null ? "SQL NULL (i.e. no value)" : expectedValue.toString(); + String actualString = actualValue == null ? "SQL NULL (i.e. no value)" : actualValue.toString(); + return " For " + fieldname + ", expected " + expectedString + " but got " + actualString + "\n"; + } + + // These asFoo methods are used for sorting records, so their defaults are intended to make broken + // records stand out. + private static String asString(JsonNode node) { + if (node == null || node.isNull()) { + return ""; + } else if (node.isTextual()) { + return node.asText(); + } else { + return Jsons.serialize(node); + } + } + + private static double asDouble(JsonNode node) { + if (node == null || !node.isNumber()) { + return Double.MIN_VALUE; + } else { + return node.longValue(); + } + } + + private static long asInt(JsonNode node) { + if (node == null || !node.isIntegralNumber()) { + return Long.MIN_VALUE; + } else { + return node.longValue(); + } + } + + private static boolean asBoolean(JsonNode node) { + if (node == null || !node.isBoolean()) { + return false; + } else { + return node.asBoolean(); + } + } + + private static Instant asTimestampWithTimezone(JsonNode node) { + if (node == null || !node.isTextual()) { + return Instant.ofEpochMilli(Long.MIN_VALUE); + } else { + try { + return Instant.parse(node.asText()); + } catch (Exception e) { + return Instant.ofEpochMilli(Long.MIN_VALUE); + } + } + } + + private static LocalDateTime asTimestampWithoutTimezone(JsonNode node) { + if (node == null || !node.isTextual()) { + return LocalDateTime.ofInstant(Instant.ofEpochMilli(Long.MIN_VALUE), ZoneOffset.UTC); + } else { + try { + return LocalDateTime.parse(node.asText()); + } catch (Exception e) { + return LocalDateTime.ofInstant(Instant.ofEpochMilli(Long.MIN_VALUE), ZoneOffset.UTC); + } + } + } + + private static OffsetTime asTimeWithTimezone(JsonNode node) { + if (node == null || !node.isTextual()) { + return OffsetTime.of(0, 0, 0, 0, ZoneOffset.UTC); + } else { + return OffsetTime.parse(node.asText()); + } + } + + private static LocalTime asTimeWithoutTimezone(JsonNode node) { + if (node == null || !node.isTextual()) { + return LocalTime.of(0, 0, 0); + } else { + try { + return LocalTime.parse(node.asText()); + } catch (Exception e) { + return LocalTime.of(0, 0, 0); + } + } + } + + private static LocalDate asDate(JsonNode node) { + if (node == null || !node.isTextual()) { + return LocalDate.ofInstant(Instant.ofEpochMilli(Long.MIN_VALUE), ZoneOffset.UTC); + } else { + try { + return LocalDate.parse(node.asText()); + } catch (Exception e) { + return LocalDate.ofInstant(Instant.ofEpochMilli(Long.MIN_VALUE), ZoneOffset.UTC); + } + } + } + + // Generics? Never heard of 'em. (I'm sorry) + private static Comparable extract(JsonNode node, String field, AirbyteType type) { + if (type instanceof AirbyteProtocolType t) { + return switch (t) { + case STRING -> asString(node.get(field)); + case NUMBER -> asDouble(node.get(field)); + case INTEGER -> asInt(node.get(field)); + case BOOLEAN -> asBoolean(node.get(field)); + case TIMESTAMP_WITH_TIMEZONE -> asTimestampWithTimezone(node.get(field)); + case TIMESTAMP_WITHOUT_TIMEZONE -> asTimestampWithoutTimezone(node.get(field)); + case TIME_WITH_TIMEZONE -> asTimeWithTimezone(node.get(field)); + case TIME_WITHOUT_TIMEZONE -> asTimeWithoutTimezone(node.get(field)); + case DATE -> asDate(node.get(field)); + case UNKNOWN -> node.toString(); + }; + } else { + return node.toString(); + } + } + +} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/schema.json b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/schema.json new file mode 100644 index 000000000000..e391324deaf7 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/schema.json @@ -0,0 +1,29 @@ +{ + "type": "object", + "properties": { + "id1": { "type": "integer" }, + "id2": { "type": "integer" }, + "updated_at": { + "type": "string", + "airbyte_type": "timestamp_with_timezone" + }, + "_ab_cdc_deleted_at": { + "type": "string", + "airbyte_type": "timestamp_with_timezone" + }, + "name": { "type": "string" }, + "address": { + "type": "object", + "properties": { + "city": { "type": "string" }, + "state": { "type": "string" } + } + }, + "age": { "type": "integer" }, + "registration_date": { + "type": "string", + "format": "date", + "airbyte_type": "date" + } + } +} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_expectedrecords_dedup_final.jsonl b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_expectedrecords_dedup_final.jsonl new file mode 100644 index 000000000000..e456f48d443a --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_expectedrecords_dedup_final.jsonl @@ -0,0 +1,4 @@ +// Keep the Alice record with more recent updated_at +{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"errors":[]}, "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": {"errors":[]}, "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": {"errors":["Problem with `age`", "Problem with `registration_date`"]}, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie"} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_expectedrecords_dedup_raw.jsonl b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_expectedrecords_dedup_raw.jsonl new file mode 100644 index 000000000000..88411c9e4de3 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_expectedrecords_dedup_raw.jsonl @@ -0,0 +1,4 @@ +// Keep the Alice record with more recent updated_at +{"_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_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_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": "this is not an integer", "registration_date": "this is not a date"}} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_expectedrecords_nondedup_final.jsonl b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_expectedrecords_nondedup_final.jsonl new file mode 100644 index 000000000000..623527f41e75 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_expectedrecords_nondedup_final.jsonl @@ -0,0 +1,5 @@ +{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"errors":[]}, "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": {"errors":[]}, "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": {"errors":[]}, "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": {"errors":["Problem with `age`", "Problem with `registration_date`"]}, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie"} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_expectedrecords_nondedup_raw.jsonl b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_expectedrecords_nondedup_raw.jsonl new file mode 100644 index 000000000000..4b4db08115e5 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_expectedrecords_nondedup_raw.jsonl @@ -0,0 +1,6 @@ +{"_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"}}} +// Note the duplicate record. In this sync mode, we don't dedup anything. +{"_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_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"}}} +// Invalid data is still allowed in the raw table. +{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": "this is not an integer", "registration_date": "this is not a date"}} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_messages.jsonl b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_messages.jsonl new file mode 100644 index 000000000000..4c5dec1a24ea --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync1_messages.jsonl @@ -0,0 +1,12 @@ +// emitted_at:1000 is equal to 1970-01-01 00:00:01Z, which is what you'll see in the expected records. +// This obviously makes no sense in relation to updated_at being in the year 2000, but that's OK +// because (from destinations POV) updated_at has no relation to emitted_at. +{"type": "RECORD", "record": {"emitted_at": 1000, "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"}}}} +// Emit a second record for id=(1,200) with a different updated_at. This generally doesn't happen +// in full refresh syncs - but if T+D is implemented correctly, it shouldn't matter +// (i.e. both records should be written to the final table). +{"type": "RECORD", "record": {"emitted_at": 1000, "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"}}}} +// Emit a record with no _ab_cdc_deleted_at field. CDC sources typically emit an explicit null, but we should handle both cases. +{"type": "RECORD", "record": {"emitted_at": 1000, "data": {"id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}}}} +// Emit a record with an invalid age. +{"type": "RECORD", "record": {"emitted_at": 1000, "data": {"id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": "this is not an integer", "registration_date": "this is not a date"}}} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_fullrefresh_append_final.jsonl b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_fullrefresh_append_final.jsonl new file mode 100644 index 000000000000..2e935f18f357 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_fullrefresh_append_final.jsonl @@ -0,0 +1,8 @@ +{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"errors":[]}, "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": {"errors":[]}, "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": {"errors":[]}, "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": {"errors":["Problem with `age`", "Problem with `registration_date`"]}, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie"} + +{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta":{"errors":[]}, "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":{"errors":[]}, "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":{"errors":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01:00:00:00Z"} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_fullrefresh_append_raw.jsonl b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_fullrefresh_append_raw.jsonl new file mode 100644 index 000000000000..5cf2a7f389ce --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_fullrefresh_append_raw.jsonl @@ -0,0 +1,9 @@ +// 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_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_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_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": "this is not an integer", "registration_date": "this is not a date"}} +// 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_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_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-01:00:00:00Z"}} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_fullrefresh_overwrite_final.jsonl b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_fullrefresh_overwrite_final.jsonl new file mode 100644 index 000000000000..0c06d6b00117 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_fullrefresh_overwrite_final.jsonl @@ -0,0 +1,3 @@ +{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta":{"errors":[]}, "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":{"errors":[]}, "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":{"errors":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01:00:00:00Z"} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_fullrefresh_overwrite_raw.jsonl b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_fullrefresh_overwrite_raw.jsonl new file mode 100644 index 000000000000..79554272b9a6 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_fullrefresh_overwrite_raw.jsonl @@ -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_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_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-01:00:00:00Z"}} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_incremental_dedup_final.jsonl b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_incremental_dedup_final.jsonl new file mode 100644 index 000000000000..10cd001e22f6 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_incremental_dedup_final.jsonl @@ -0,0 +1,3 @@ +{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta":{"errors":[]}, "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": {"errors":["Problem with `age`", "Problem with `registration_date`"]}, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie"} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_incremental_dedup_raw.jsonl b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_incremental_dedup_raw.jsonl new file mode 100644 index 000000000000..bd79da0ea871 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_expectedrecords_incremental_dedup_raw.jsonl @@ -0,0 +1,5 @@ +{"_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"}}} +// Keep the record that deleted Bob, but delete the other records associated with id=(1, 201) +{"_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-01:00:00:00Z"}} +// And keep Charlie's record, even though it wasn't reemitted in sync2. +{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": "this is not an integer", "registration_date": "this is not a date"}} diff --git a/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_messages.jsonl b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_messages.jsonl new file mode 100644 index 000000000000..1f828f31f5d3 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping-test/src/main/resources/sync2_messages.jsonl @@ -0,0 +1,5 @@ +{"type": "RECORD", "record": {"emitted_at": 2000, "data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}}} +{"type": "RECORD", "record": {"emitted_at": 2000, "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"}}}} +// Set deleted_at to something non-null. Again, T+D doesn't check the actual _value_ of deleted_at (i.e. the fact that it's in the past is irrelevant). +// It only cares whether deleted_at is non-null. So this should delete Bob from the final table (in dedup mode). +{"type": "RECORD", "record": {"emitted_at": 2000, "data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01:00:00:00Z"}}} diff --git a/airbyte-integrations/bases/base-typing-deduping/build.gradle b/airbyte-integrations/bases/base-typing-deduping/build.gradle new file mode 100644 index 000000000000..1381b8b45801 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping/build.gradle @@ -0,0 +1,7 @@ +plugins { + id 'java-library' +} + +dependencies { + implementation libs.airbyte.protocol +} diff --git a/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/AirbyteType.java b/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/AirbyteType.java new file mode 100644 index 000000000000..4351bcc6cc40 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/AirbyteType.java @@ -0,0 +1,193 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.base.destination.typing_deduping; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.AirbyteProtocolType; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.Array; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.OneOf; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.Struct; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.UnsupportedOneOf; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public sealed interface AirbyteType permits Array,OneOf,Struct,UnsupportedOneOf,AirbyteProtocolType { + + Logger LOGGER = LoggerFactory.getLogger(AirbyteTypeUtils.class); + + /** + * The most common call pattern is probably to use this method on the stream schema, verify that + * it's an {@link Struct} schema, and then call {@link Struct#properties()} to get the columns. + *

+ * If the top-level schema is not an object, then we can't really do anything with it, and should + * probably fail the sync. (but see also {@link OneOf#asColumns()}). + */ + static AirbyteType fromJsonSchema(final JsonNode schema) { + try { + final JsonNode topLevelType = schema.get("type"); + if (topLevelType != null) { + if (topLevelType.isTextual()) { + if (AirbyteTypeUtils.nodeIsType(topLevelType, "object")) { + return getStruct(schema); + } else if (AirbyteTypeUtils.nodeIsType(topLevelType, "array")) { + return getArray(schema); + } + } else if (topLevelType.isArray()) { + final List typeOptions = new ArrayList<>(); + topLevelType.elements().forEachRemaining(element -> { + // ignore "null" type and remove duplicates + String type = element.asText(""); + if (!"null".equals(type) && !typeOptions.contains(type)) { + typeOptions.add(element.asText()); + } + }); + + // we encounter an array of types that actually represents a single type rather than a OneOf + if (typeOptions.size() == 1) { + if (typeOptions.get(0).equals("object")) { + return getStruct(schema); + } else if (typeOptions.get(0).equals("array")) { + return getArray(schema); + } else { + return AirbyteTypeUtils.getAirbyteProtocolType(schema); + } + } + + final List options = typeOptions.stream().map(typeOption -> { + // Recurse into a schema that forces a specific one of each option + JsonNode schemaClone = schema.deepCopy(); + // schema is guaranteed to be an object here, because we know it has a `type` key + ((ObjectNode) schemaClone).put("type", typeOption); + return fromJsonSchema(schemaClone); + }).toList(); + return new OneOf(options); + } + } else if (schema.hasNonNull("oneOf")) { + final List options = new ArrayList<>(); + schema.get("oneOf").elements().forEachRemaining(element -> options.add(fromJsonSchema(element))); + return new UnsupportedOneOf(options); + } else if (schema.hasNonNull("properties")) { + // The schema has neither type nor oneof, but it does have properties. Assume we're looking at a + // struct. + // This is for backwards-compatibility with legacy normalization. + return getStruct(schema); + } + return AirbyteTypeUtils.getAirbyteProtocolType(schema); + } catch (final Exception e) { + LOGGER.error("Exception parsing JSON schema {}: {}; returning UNKNOWN.", schema, e); + return AirbyteProtocolType.UNKNOWN; + } + } + + private static Struct getStruct(final JsonNode schema) { + final LinkedHashMap propertiesMap = new LinkedHashMap<>(); + final JsonNode properties = schema.get("properties"); + if (properties != null) { + properties.fields().forEachRemaining(property -> { + final String key = property.getKey(); + final JsonNode value = property.getValue(); + propertiesMap.put(key, fromJsonSchema(value)); + }); + } + return new Struct(propertiesMap); + } + + private static Array getArray(final JsonNode schema) { + final JsonNode items = schema.get("items"); + if (items == null) { + return new Array(AirbyteProtocolType.UNKNOWN); + } else { + return new Array(fromJsonSchema(items)); + } + } + + enum AirbyteProtocolType implements AirbyteType { + + STRING, + NUMBER, + INTEGER, + BOOLEAN, + TIMESTAMP_WITH_TIMEZONE, + TIMESTAMP_WITHOUT_TIMEZONE, + TIME_WITH_TIMEZONE, + TIME_WITHOUT_TIMEZONE, + DATE, + UNKNOWN; + + public static AirbyteProtocolType matches(final String type) { + try { + return AirbyteProtocolType.valueOf(type.toUpperCase()); + } catch (final IllegalArgumentException e) { + LOGGER.error(String.format("Could not find matching AirbyteProtocolType for \"%s\": %s", type, e)); + return UNKNOWN; + } + } + + } + + /** + * @param properties Use LinkedHashMap to preserve insertion order. + */ + record Struct(LinkedHashMap properties) implements AirbyteType { + + } + + record Array(AirbyteType items) implements AirbyteType { + + } + + /** + * Represents a {oneOf: [...]} schema. + *

+ * This is purely a legacy type that we should eventually delete. See also {@link OneOf}. + */ + record UnsupportedOneOf(List options) implements AirbyteType { + + } + + /** + * Represents a {type: [a, b, ...]} schema. This is theoretically equivalent to {oneOf: [{type: a}, + * {type: b}, ...]} but legacy normalization only handles the {type: [...]} schemas. + *

+ * Eventually we should: + *

    + *
  1. Announce a breaking change to handle both oneOf styles the same
  2. + *
  3. Test against some number of API sources to verify that they won't break badly
  4. + *
  5. Update {@link AirbyteType#fromJsonSchema(JsonNode)} to parse both styles into + * SupportedOneOf
  6. + *
  7. Delete UnsupportedOneOf
  8. + *
+ */ + record OneOf(List options) implements AirbyteType { + + /** + * This is a hack to handle weird schemas like {type: [object, string]}. If a stream's top-level + * schema looks like this, we still want to be able to extract the object properties (i.e. treat it + * as though the string option didn't exist). + * + * @throws IllegalArgumentException if we cannot extract columns from this schema + */ + public LinkedHashMap asColumns() { + final long numObjectOptions = options.stream().filter(o -> o instanceof Struct).count(); + if (numObjectOptions > 1) { + LOGGER.error("Can't extract columns from a schema with multiple object options"); + return new LinkedHashMap<>(); + } + + return (options.stream().filter(o -> o instanceof Struct).findFirst()) + .map(o -> ((Struct) o).properties()) + .orElseGet(() -> { + LOGGER.error("Can't extract columns from a schema with no object options"); + return new LinkedHashMap<>(); + }); + } + + } + +} diff --git a/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/AirbyteTypeUtils.java b/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/AirbyteTypeUtils.java new file mode 100644 index 000000000000..9c98f83caac7 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/AirbyteTypeUtils.java @@ -0,0 +1,156 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.base.destination.typing_deduping; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.TextNode; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.AirbyteProtocolType; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.Array; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.OneOf; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.Struct; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class AirbyteTypeUtils { + + private static final Logger LOGGER = LoggerFactory.getLogger(AirbyteTypeUtils.class); + + // Map from a protocol type to what other protocol types should take precedence over it if present + // in a OneOf + private static final Map> EXCLUDED_PROTOCOL_TYPES_MAP = ImmutableMap.of( + AirbyteProtocolType.BOOLEAN, ImmutableList.of(AirbyteProtocolType.STRING, AirbyteProtocolType.NUMBER, AirbyteProtocolType.INTEGER), + AirbyteProtocolType.INTEGER, ImmutableList.of(AirbyteProtocolType.STRING, AirbyteProtocolType.NUMBER), + AirbyteProtocolType.NUMBER, ImmutableList.of(AirbyteProtocolType.STRING)); + + // Protocol types in order of precedence + private static final List ORDERED_PROTOCOL_TYPES = ImmutableList.of( + AirbyteProtocolType.BOOLEAN, + AirbyteProtocolType.INTEGER, + AirbyteProtocolType.NUMBER, + AirbyteProtocolType.TIMESTAMP_WITHOUT_TIMEZONE, + AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE, + AirbyteProtocolType.DATE, + AirbyteProtocolType.TIME_WITH_TIMEZONE, + AirbyteProtocolType.TIME_WITHOUT_TIMEZONE, + AirbyteProtocolType.STRING); + + protected static boolean nodeIsType(final JsonNode node, final String type) { + if (node == null || !node.isTextual()) { + return false; + } + return node.equals(TextNode.valueOf(type)); + } + + private static boolean nodeIsOrContainsType(final JsonNode node, final String type) { + if (node == null) { + return false; + } else if (node.isTextual()) { + return nodeIsType(node, type); + } else if (node.isArray()) { + for (final JsonNode element : node) { + if (nodeIsType(element, type)) { + return true; + } + } + } + return false; + } + + protected static AirbyteType getAirbyteProtocolType(final JsonNode node) { + if (node.isTextual()) { + return AirbyteProtocolType.matches(node.asText()); + } + + final JsonNode propertyType = node.get("type"); + final JsonNode airbyteType = node.get("airbyte_type"); + final JsonNode format = node.get("format"); + + if (nodeIsOrContainsType(propertyType, "boolean")) { + return AirbyteProtocolType.BOOLEAN; + } else if (nodeIsOrContainsType(propertyType, "integer")) { + return AirbyteProtocolType.INTEGER; + } else if (nodeIsOrContainsType(propertyType, "number")) { + if (nodeIsType(airbyteType, "integer")) { + return AirbyteProtocolType.INTEGER; + } else { + return AirbyteProtocolType.NUMBER; + } + } else if (nodeIsOrContainsType(propertyType, "string")) { + if (nodeIsOrContainsType(format, "date")) { + return AirbyteProtocolType.DATE; + } else if (nodeIsType(format, "time")) { + if (nodeIsType(airbyteType, "timestamp_without_timezone")) { + return AirbyteProtocolType.TIME_WITHOUT_TIMEZONE; + } else if (nodeIsType(airbyteType, "timestamp_with_timezone")) { + return AirbyteProtocolType.TIME_WITH_TIMEZONE; + } + } else if (nodeIsOrContainsType(format, "date-time")) { + if (nodeIsType(airbyteType, "timestamp_without_timezone")) { + return AirbyteProtocolType.TIMESTAMP_WITHOUT_TIMEZONE; + } else if (airbyteType == null || nodeIsType(airbyteType, "timestamp_with_timezone")) { + return AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE; + } + } else { + return AirbyteProtocolType.STRING; + } + } + + return AirbyteProtocolType.UNKNOWN; + } + + // Pick which type in a OneOf has precedence + public static AirbyteType chooseOneOfType(final OneOf o) { + final List options = o.options(); + + // record what types are present + Array foundArrayType = null; + Struct foundStructType = null; + final Map typePresenceMap = new HashMap<>(); + Arrays.stream(AirbyteProtocolType.values()).map(type -> typePresenceMap.put(type, false)); + + // looping through the options only once for efficiency + for (final AirbyteType option : options) { + if (option instanceof final Array a) { + foundArrayType = a; + } else if (option instanceof final Struct s) { + foundStructType = s; + } else if (option instanceof final AirbyteProtocolType p) { + typePresenceMap.put(p, true); + } + } + + if (foundArrayType != null) { + return foundArrayType; + } else if (foundStructType != null) { + return foundStructType; + } else { + for (final AirbyteProtocolType protocolType : ORDERED_PROTOCOL_TYPES) { + if (typePresenceMap.getOrDefault(protocolType, false)) { + boolean foundExcludedTypes = false; + final List excludedTypes = EXCLUDED_PROTOCOL_TYPES_MAP.getOrDefault(protocolType, Collections.emptyList()); + for (final AirbyteProtocolType excludedType : excludedTypes) { + if (typePresenceMap.getOrDefault(excludedType, false)) { + foundExcludedTypes = true; + break; + } + } + if (!foundExcludedTypes) { + return protocolType; + } + } + } + } + + return AirbyteProtocolType.UNKNOWN; + } + +} diff --git a/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/CatalogParser.java b/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/CatalogParser.java new file mode 100644 index 000000000000..65137d1e4f4c --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/CatalogParser.java @@ -0,0 +1,158 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.base.destination.typing_deduping; + +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.Struct; +import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator.ColumnId; +import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator.StreamId; +import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog; +import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream; +import io.airbyte.protocol.models.v0.DestinationSyncMode; +import io.airbyte.protocol.models.v0.SyncMode; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map.Entry; +import java.util.Optional; +import org.apache.commons.codec.digest.DigestUtils; + +public class CatalogParser { + + public static final String DEFAULT_RAW_TABLE_NAMESPACE = "airbyte"; + private final SqlGenerator sqlGenerator; + private final String rawNamespaceOverride; + + public CatalogParser(final SqlGenerator sqlGenerator) { + this(sqlGenerator, DEFAULT_RAW_TABLE_NAMESPACE); + } + + public CatalogParser(final SqlGenerator sqlGenerator, String rawNamespaceOverride) { + this.sqlGenerator = sqlGenerator; + this.rawNamespaceOverride = rawNamespaceOverride; + } + + public record ParsedCatalog(List streams) { + + public StreamConfig getStream(String namespace, String name) { + return streams.stream() + .filter(s -> s.id().originalNamespace().equals(namespace) && s.id().originalName().equals(name)) + .findFirst() + .orElseThrow(() -> new IllegalArgumentException(String.format( + "Could not find stream %s.%s out of streams %s", + namespace, + name, + streams.stream().map(stream -> stream.id().originalNamespace() + "." + stream.id().originalName()).toList()))); + } + + } + + public record StreamConfig(StreamId id, + SyncMode syncMode, + DestinationSyncMode destinationSyncMode, + List primaryKey, + Optional cursor, + LinkedHashMap columns) { + + } + + public ParsedCatalog parseCatalog(ConfiguredAirbyteCatalog catalog) { + // this code is bad and I feel bad + // it's mostly a port of the old normalization logic to prevent tablename collisions. + // tbh I have no idea if it works correctly. + final List streamConfigs = new ArrayList<>(); + for (ConfiguredAirbyteStream stream : catalog.getStreams()) { + final StreamConfig originalStreamConfig = toStreamConfig(stream); + // Use empty string quote because we don't really care + if (streamConfigs.stream().anyMatch(s -> s.id().finalTableId("").equals(originalStreamConfig.id().finalTableId(""))) + || streamConfigs.stream().anyMatch(s -> s.id().rawTableId("").equals(originalStreamConfig.id().rawTableId("")))) { + String originalNamespace = stream.getStream().getNamespace(); + String originalName = stream.getStream().getName(); + // ... this logic is ported from legacy normalization, and maybe should change? + // We're taking a hash of the quoted namespace and the unquoted stream name + final String hash = DigestUtils.sha1Hex(originalStreamConfig.id().finalNamespace() + "&airbyte&" + originalName).substring(0, 3); + final String newName = originalName + "_" + hash; + streamConfigs.add(new StreamConfig( + sqlGenerator.buildStreamId(originalNamespace, newName, rawNamespaceOverride), + originalStreamConfig.syncMode(), + originalStreamConfig.destinationSyncMode(), + originalStreamConfig.primaryKey(), + originalStreamConfig.cursor(), + originalStreamConfig.columns())); + } else { + streamConfigs.add(originalStreamConfig); + } + } + return new ParsedCatalog(streamConfigs); + } + + private StreamConfig toStreamConfig(ConfiguredAirbyteStream stream) { + AirbyteType schema = AirbyteType.fromJsonSchema(stream.getStream().getJsonSchema()); + LinkedHashMap airbyteColumns; + if (schema instanceof Struct o) { + airbyteColumns = o.properties(); + } else if (schema instanceof AirbyteType.OneOf o) { + airbyteColumns = o.asColumns(); + } else { + throw new IllegalArgumentException("Top-level schema must be an object"); + } + + if (stream.getPrimaryKey().stream().anyMatch(key -> key.size() > 1)) { + throw new IllegalArgumentException("Only top-level primary keys are supported"); + } + final List primaryKey = stream.getPrimaryKey().stream().map(key -> sqlGenerator.buildColumnId(key.get(0))).toList(); + + if (stream.getCursorField().size() > 1) { + throw new IllegalArgumentException("Only top-level cursors are supported"); + } + final Optional cursor; + if (stream.getCursorField().size() > 0) { + cursor = Optional.of(sqlGenerator.buildColumnId(stream.getCursorField().get(0))); + } else { + cursor = Optional.empty(); + } + + // this code is really bad and I'm not convinced we need to preserve this behavior. + // as with the tablename collisions thing above - we're trying to preserve legacy normalization's + // naming conventions here. + final LinkedHashMap columns = new LinkedHashMap<>(); + for (Entry entry : airbyteColumns.entrySet()) { + ColumnId originalColumnId = sqlGenerator.buildColumnId(entry.getKey()); + ColumnId columnId; + if (columns.keySet().stream().noneMatch(c -> c.canonicalName().equals(originalColumnId.canonicalName()))) { + // None of the existing columns have the same name. We can add this new column as-is. + columnId = originalColumnId; + } else { + // One of the existing columns has the same name. We need to handle this collision. + // Append _1, _2, _3, ... to the column name until we find one that doesn't collide. + int i = 1; + while (true) { + columnId = sqlGenerator.buildColumnId(entry.getKey() + "_" + i); + String canonicalName = columnId.canonicalName(); + if (columns.keySet().stream().noneMatch(c -> c.canonicalName().equals(canonicalName))) { + break; + } else { + i++; + } + } + // But we need to keep the original name so that we can still fetch it out of the JSON records. + columnId = new ColumnId( + columnId.name(), + originalColumnId.originalName(), + columnId.canonicalName()); + } + + columns.put(columnId, entry.getValue()); + } + + return new StreamConfig( + sqlGenerator.buildStreamId(stream.getStream().getNamespace(), stream.getStream().getName(), rawNamespaceOverride), + stream.getSyncMode(), + stream.getDestinationSyncMode(), + primaryKey, + cursor, + columns); + } + +} diff --git a/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/SqlGenerator.java b/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/SqlGenerator.java new file mode 100644 index 000000000000..b6c14d31ef18 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/SqlGenerator.java @@ -0,0 +1,126 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.base.destination.typing_deduping; + +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser.StreamConfig; +import java.util.Optional; + +public interface SqlGenerator { + + /** + * In general, callers should not directly instantiate this class. Use + * {@link #buildStreamId(String, String, String)} instead. + *

+ * All names/namespaces are intended to be quoted, but do not explicitly contain quotes. For + * example, finalName might be "foo bar"; the caller is required to wrap that in quotes before using + * it in a query. + * + * @param finalNamespace the namespace where the final table will be created + * @param finalName the name of the final table + * @param rawNamespace the namespace where the raw table will be created (typically "airbyte") + * @param rawName the name of the raw table (typically namespace_name, but may be different if there + * are collisions). There is no rawNamespace because we assume that we're writing raw tables + * to the airbyte namespace. + */ + record StreamId(String finalNamespace, String finalName, String rawNamespace, String rawName, String originalNamespace, String originalName) { + + /** + * Most databases/warehouses use a `schema.name` syntax to identify tables. This is a convenience + * method to generate that syntax. + */ + public String finalTableId(String quote) { + return quote + finalNamespace + quote + "." + quote + finalName + quote; + } + + public String finalTableId(String suffix, String quote) { + return quote + finalNamespace + quote + "." + quote + finalName + suffix + quote; + } + + public String rawTableId(String quote) { + return quote + rawNamespace + quote + "." + quote + rawName + quote; + } + + public String finalName(final String quote) { + return quote + finalName + quote; + } + + public String finalNamespace(final String quote) { + return quote + finalNamespace + quote; + } + + } + + /** + * In general, callers should not directly instantiate this class. Use + * {@link #buildColumnId(String)} instead. + * + * @param name the name of the column in the final table. Callers should prefer + * {@link #name(String)} when using the column in a query. + * @param originalName the name of the field in the raw JSON blob + * @param canonicalName the name of the field according to the destination. Used for deduping. + * Useful if a destination warehouse handles columns ignoring case, but preserves case in the + * table schema. + */ + record ColumnId(String name, String originalName, String canonicalName) { + + public String name(final String quote) { + return quote + name + quote; + } + + } + + StreamId buildStreamId(String namespace, String name, String rawNamespaceOverride); + + ColumnId buildColumnId(String name); + + /** + * Generate a SQL statement to create a fresh table to match the given stream. + *

+ * The generated SQL may throw an exception if the table already exists. Callers should use + * {@link #alterTable(StreamConfig, java.lang.Object)} if the table is known to exist. + * + * @param suffix A suffix to add to the stream name. Useful for full refresh overwrite syncs, where + * we write the entire sync to a temp table. + */ + String createTable(final StreamConfig stream, final String suffix); + + /** + * Generate a SQL statement to alter an existing table to match the given stream. + *

+ * The operations may differ based on the existing table definition (BigQuery does not allow + * altering a partitioning scheme and requires you to recreate+rename the table; snowflake only + * allows altering some column types to certain other types, etc.). + */ + String alterTable(final StreamConfig stream, DialectTableDefinition existingTable); + + /** + * Generate a SQL statement to copy new data from the raw table into the final table. + *

+ * Supports both incremental and one-shot loading. (maybe.) + *

+ * Responsible for: + *

    + *
  • Pulling new raw records from a table (i.e. records with null _airbyte_loaded_at)
  • + *
  • Extracting the JSON fields and casting to the appropriate types
  • + *
  • Handling errors in those casts
  • + *
  • Merging those typed records into an existing table
  • + *
  • Updating the raw records with SET _airbyte_loaded_at = now()
  • + *
+ *

+ * Implementing classes are recommended to break this into smaller methods, which can be tested in + * isolation. However, this interface only requires a single mega-method. + * + * @param finalSuffix the suffix of the final table to write to. If empty string, writes to the + * final table directly. Useful for full refresh overwrite syncs, where we write the entire + * sync to a temp table and then swap it into the final table at the end. + */ + String updateTable(String finalSuffix, final StreamConfig stream); + + /** + * Drop the previous final table, and rename the new final table to match the old final table. + */ + Optional overwriteFinalTable(String finalSuffix, StreamConfig stream); + +} diff --git a/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/TypeAndDedupeOperationValve.java b/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/TypeAndDedupeOperationValve.java new file mode 100644 index 000000000000..096392d47aec --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/TypeAndDedupeOperationValve.java @@ -0,0 +1,155 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.base.destination.typing_deduping; + +import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Supplier; + +/** + * A slightly more complicated way to keep track of when to perform type and dedupe operations per + * stream + */ +public class TypeAndDedupeOperationValve extends ConcurrentHashMap { + + private static final long TWO_MINUTES_MILLIS = 1000 * 60 * 2; + + private static final long FIVE_MINUTES_MILLIS = 1000 * 60 * 5; + + private static final long TEN_MINUTES_MILLIS = 1000 * 60 * 10; + + // 15 minutes is the maximum amount of time allowed between checkpoints as defined by + // The Airbyte Protocol + private static final long FIFTEEN_MINUTES_MILLIS = 1000 * 60 * 15; + + // New users of airbyte likely want to see data flowing into their tables as soon as possible + // However, as their destination tables grow in size, typing and de-duping data becomes an expensive + // operation + // To strike a balance between showing data quickly and not slowing down the entire sync, we use an + // increasing + // interval based approach. This is not fancy, just hard coded intervals. + private static final List typeAndDedupeIncreasingIntervals = List.of( + TWO_MINUTES_MILLIS, + FIVE_MINUTES_MILLIS, + TEN_MINUTES_MILLIS, + FIFTEEN_MINUTES_MILLIS); + + // Constantly getting the system time adds a bit of overhead, adding a minimum record count + // To reduce calls for system time + private static final int MINIMUM_RECORD_INTERVAL = 100; + + private static final Supplier SYSTEM_NOW = () -> System.currentTimeMillis(); + + private ConcurrentHashMap incrementalIndex; + + private final Supplier nowness; + private ConcurrentHashMap recordCounts; + + public TypeAndDedupeOperationValve() { + this(SYSTEM_NOW); + } + + /** + * This constructor is here because mocking System.currentTimeMillis() is a pain :( + * + * @param nownessSupplier Supplier which will return a long value representing now + */ + public TypeAndDedupeOperationValve(Supplier nownessSupplier) { + super(); + incrementalIndex = new ConcurrentHashMap<>(); + recordCounts = new ConcurrentHashMap<>(); + this.nowness = nownessSupplier; + } + + @Override + public Long put(final AirbyteStreamNameNamespacePair key, final Long value) { + if (!incrementalIndex.containsKey(key)) { + incrementalIndex.put(key, 0); + } + if (!recordCounts.containsKey(key)) { + recordCounts.put(key, 1l); + } + return super.put(key, value); + + } + + /** + * Adds a stream specific timestamp to track type and dedupe operations + * + * @param key the AirbyteStreamNameNamespacePair to track + */ + public void addStream(final AirbyteStreamNameNamespacePair key) { + put(key, nowness.get()); + } + + /** + * Whether we should type and dedupe at this point in time for this particular stream. + * + * @param key the stream in question + * @return a boolean indicating whether we have crossed the interval threshold for typing and + * deduping. + */ + public boolean readyToTypeAndDedupeWithAdditionalRecord(final AirbyteStreamNameNamespacePair key) { + return false; + // if (!containsKey(key)) { + // return false; + // } + // recordCounts.put(key, recordCounts.get(key) + 1); + // if (recordCounts.get(key) % MINIMUM_RECORD_INTERVAL == 0) { + // return nowness.get() - get(key) > + // typeAndDedupeIncreasingIntervals.get(incrementalIndex.get(key)); + // } + // return false; + } + + /** + * Increment the interval at which typing and deduping should occur for the stream, max out at last + * index of {@link TypeAndDedupeOperationValve#typeAndDedupeIncreasingIntervals} + * + * @param key the stream to increment the interval of + * @return the index of the typing and deduping interval associated with this stream + */ + public int incrementInterval(final AirbyteStreamNameNamespacePair key) { + if (incrementalIndex.get(key) < typeAndDedupeIncreasingIntervals.size() - 1) { + incrementalIndex.put(key, incrementalIndex.get(key) + 1); + } + return incrementalIndex.get(key); + } + + /** + * Meant to be called after + * {@link TypeAndDedupeOperationValve#readyToTypeAndDedupeWithAdditionalRecord(AirbyteStreamNameNamespacePair)} + * will set a streams last operation to the current time and increase its index reference in + * {@link TypeAndDedupeOperationValve#typeAndDedupeIncreasingIntervals} + * + * @param key the stream to update + */ + public void updateTimeAndIncreaseInterval(final AirbyteStreamNameNamespacePair key) { + put(key, nowness.get()); + incrementInterval(key); + } + + /** + * Get the current interval for the stream + * + * @param key the stream in question + * @return a long value representing the length of the interval milliseconds + */ + public Long getIncrementInterval(final AirbyteStreamNameNamespacePair key) { + return typeAndDedupeIncreasingIntervals.get(incrementalIndex.get(key)); + } + + /** + * Get the current record count per stream + * + * @param key the stream in question + * @return the recrod count + */ + public Long getRecordCount(final AirbyteStreamNameNamespacePair key) { + return recordCounts.get(key); + } + +} diff --git a/airbyte-integrations/bases/base-typing-deduping/src/test/java/io/airbyte/integrations/base/destination/typing_deduping/AirbyteTypeTest.java b/airbyte-integrations/bases/base-typing-deduping/src/test/java/io/airbyte/integrations/base/destination/typing_deduping/AirbyteTypeTest.java new file mode 100644 index 000000000000..d562eb5f365a --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping/src/test/java/io/airbyte/integrations/base/destination/typing_deduping/AirbyteTypeTest.java @@ -0,0 +1,510 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.base.destination.typing_deduping; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableList; +import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.AirbyteProtocolType; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.Array; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.OneOf; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.Struct; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.UnsupportedOneOf; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import org.junit.jupiter.api.Test; + +public class AirbyteTypeTest { + + @Test + public void testStruct() { + final List structSchema = new ArrayList<>(); + structSchema.add(""" + { + "type": "object", + "properties": { + "key1": { + "type": "boolean" + }, + "key2": { + "type": "integer" + }, + "key3": { + "type": "number", + "airbyte_type": "integer" + }, + "key4": { + "type": "number" + }, + "key5": { + "type": "string", + "format": "date" + }, + "key6": { + "type": "string", + "format": "time", + "airbyte_type": "timestamp_without_timezone" + }, + "key7": { + "type": "string", + "format": "time", + "airbyte_type": "timestamp_with_timezone" + }, + "key8": { + "type": "string", + "format": "date-time", + "airbyte_type": "timestamp_without_timezone" + }, + "key9": { + "type": "string", + "format": ["date-time", "foo"], + "airbyte_type": "timestamp_with_timezone" + }, + "key10": { + "type": "string", + "format": "date-time" + }, + "key11": { + "type": "string" + } + } + } + """); + structSchema.add(""" + { + "type": ["object"], + "properties": { + "key1": { + "type": ["boolean"] + }, + "key2": { + "type": ["integer"] + }, + "key3": { + "type": ["number"], + "airbyte_type": "integer" + }, + "key4": { + "type": ["number"] + }, + "key5": { + "type": ["string"], + "format": "date" + }, + "key6": { + "type": ["string"], + "format": "time", + "airbyte_type": "timestamp_without_timezone" + }, + "key7": { + "type": ["string"], + "format": "time", + "airbyte_type": "timestamp_with_timezone" + }, + "key8": { + "type": ["string"], + "format": "date-time", + "airbyte_type": "timestamp_without_timezone" + }, + "key9": { + "type": ["string"], + "format": ["date-time", "foo"], + "airbyte_type": "timestamp_with_timezone" + }, + "key10": { + "type": ["string"], + "format": "date-time" + }, + "key11": { + "type": ["string"] + } + } + } + """); + structSchema.add(""" + { + "type": ["null", "object"], + "properties": { + "key1": { + "type": ["null", "boolean"] + }, + "key2": { + "type": ["null", "integer"] + }, + "key3": { + "type": ["null", "number"], + "airbyte_type": "integer" + }, + "key4": { + "type": ["null", "number"] + }, + "key5": { + "type": ["null", "string"], + "format": "date" + }, + "key6": { + "type": ["null", "string"], + "format": "time", + "airbyte_type": "timestamp_without_timezone" + }, + "key7": { + "type": ["null", "string"], + "format": "time", + "airbyte_type": "timestamp_with_timezone" + }, + "key8": { + "type": ["null", "string"], + "format": "date-time", + "airbyte_type": "timestamp_without_timezone" + }, + "key9": { + "type": ["null", "string"], + "format": ["date-time", "foo"], + "airbyte_type": "timestamp_with_timezone" + }, + "key10": { + "type": ["null", "string"], + "format": "date-time" + }, + "key11": { + "type": ["null", "string"] + } + } + } + """); + + final LinkedHashMap propertiesMap = new LinkedHashMap<>(); + propertiesMap.put("key1", AirbyteProtocolType.BOOLEAN); + propertiesMap.put("key2", AirbyteProtocolType.INTEGER); + propertiesMap.put("key3", AirbyteProtocolType.INTEGER); + propertiesMap.put("key4", AirbyteProtocolType.NUMBER); + propertiesMap.put("key5", AirbyteProtocolType.DATE); + propertiesMap.put("key6", AirbyteProtocolType.TIME_WITHOUT_TIMEZONE); + propertiesMap.put("key7", AirbyteProtocolType.TIME_WITH_TIMEZONE); + propertiesMap.put("key8", AirbyteProtocolType.TIMESTAMP_WITHOUT_TIMEZONE); + propertiesMap.put("key9", AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE); + propertiesMap.put("key10", AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE); + propertiesMap.put("key11", AirbyteProtocolType.STRING); + + final AirbyteType struct = new Struct(propertiesMap); + for (final String schema : structSchema) { + assertEquals(struct, AirbyteType.fromJsonSchema(Jsons.deserialize(schema))); + } + } + + @Test + public void testEmptyStruct() { + final List structSchema = new ArrayList<>(); + structSchema.add(""" + { + "type": "object" + } + """); + structSchema.add(""" + { + "type": ["object"] + } + """); + structSchema.add(""" + { + "type": ["null", "object"] + } + """); + + final AirbyteType struct = new Struct(new LinkedHashMap<>()); + for (final String schema : structSchema) { + assertEquals(struct, AirbyteType.fromJsonSchema(Jsons.deserialize(schema))); + } + } + + @Test + public void testImplicitStruct() { + final String structSchema = """ + { + "properties": { + "key1": { + "type": "boolean" + } + } + } + """; + + final LinkedHashMap propertiesMap = new LinkedHashMap<>(); + propertiesMap.put("key1", AirbyteProtocolType.BOOLEAN); + + final AirbyteType struct = new Struct(propertiesMap); + assertEquals(struct, AirbyteType.fromJsonSchema(Jsons.deserialize(structSchema))); + } + + @Test + public void testArray() { + final List arraySchema = new ArrayList<>(); + arraySchema.add(""" + { + "type": "array", + "items": { + "type": "string", + "format": "date-time", + "airbyte_type": "timestamp_with_timezone" + } + } + """); + arraySchema.add(""" + { + "type": ["array"], + "items": { + "type": ["string"], + "format": "date-time", + "airbyte_type": "timestamp_with_timezone" + } + } + """); + arraySchema.add(""" + { + "type": ["null", "array"], + "items": { + "type": ["null", "string"], + "format": "date-time", + "airbyte_type": "timestamp_with_timezone" + } + } + """); + + final AirbyteType array = new Array(AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE); + for (final String schema : arraySchema) { + assertEquals(array, AirbyteType.fromJsonSchema(Jsons.deserialize(schema))); + } + } + + @Test + public void testEmptyArray() { + final List arraySchema = new ArrayList<>(); + arraySchema.add(""" + { + "type": "array" + } + """); + arraySchema.add(""" + { + "type": ["array"] + } + """); + + arraySchema.add(""" + { + "type": ["null", "array"] + } + """); + + final AirbyteType array = new Array(AirbyteProtocolType.UNKNOWN); + for (final String schema : arraySchema) { + assertEquals(array, AirbyteType.fromJsonSchema(Jsons.deserialize(schema))); + } + } + + @Test + public void testUnsupportedOneOf() { + final String unsupportedOneOfSchema = """ + { + "oneOf": ["number", "string"] + } + """; + + final List options = new ArrayList<>(); + options.add(AirbyteProtocolType.NUMBER); + options.add(AirbyteProtocolType.STRING); + + final UnsupportedOneOf unsupportedOneOf = new UnsupportedOneOf(options); + assertEquals(unsupportedOneOf, AirbyteType.fromJsonSchema(Jsons.deserialize(unsupportedOneOfSchema))); + } + + @Test + public void testOneOf() { + + final String oneOfSchema = """ + { + "type": ["string", "number"] + } + """; + + final List options = new ArrayList<>(); + options.add(AirbyteProtocolType.STRING); + options.add(AirbyteProtocolType.NUMBER); + + final OneOf oneOf = new OneOf(options); + assertEquals(oneOf, AirbyteType.fromJsonSchema(Jsons.deserialize(oneOfSchema))); + } + + @Test + public void testOneOfComplex() { + JsonNode schema = Jsons.deserialize(""" + { + "type": ["string", "object", "array", "null", "string", "object", "array", "null"], + "properties": { + "foo": {"type": "string"} + }, + "items": {"type": "string"} + } + """); + + AirbyteType parsed = AirbyteType.fromJsonSchema(schema); + + AirbyteType expected = new OneOf(List.of( + AirbyteProtocolType.STRING, + new Struct(new LinkedHashMap<>() { + + { + put("foo", AirbyteProtocolType.STRING); + } + + }), + new Array(AirbyteProtocolType.STRING))); + assertEquals(expected, parsed); + } + + @Test + public void testOneOfUnderspecifiedNonPrimitives() { + JsonNode schema = Jsons.deserialize(""" + { + "type": ["string", "object", "array", "null", "string", "object", "array", "null"] + } + """); + + AirbyteType parsed = AirbyteType.fromJsonSchema(schema); + + AirbyteType expected = new OneOf(List.of( + AirbyteProtocolType.STRING, + new Struct(new LinkedHashMap<>()), + new Array(AirbyteProtocolType.UNKNOWN))); + assertEquals(expected, parsed); + } + + @Test + public void testInvalidTextualType() { + final String invalidTypeSchema = """ + { + "type": "foo" + } + """; + assertEquals(AirbyteProtocolType.UNKNOWN, AirbyteType.fromJsonSchema(Jsons.deserialize(invalidTypeSchema))); + } + + @Test + public void testInvalidBooleanType() { + final String invalidTypeSchema = """ + { + "type": true + } + """; + assertEquals(AirbyteProtocolType.UNKNOWN, AirbyteType.fromJsonSchema(Jsons.deserialize(invalidTypeSchema))); + } + + @Test + public void testInvalid() { + final List invalidSchema = new ArrayList<>(); + invalidSchema.add(""); + invalidSchema.add("null"); + invalidSchema.add("true"); + invalidSchema.add("false"); + invalidSchema.add("1"); + invalidSchema.add("\"\""); + invalidSchema.add("[]"); + invalidSchema.add("{}"); + + for (final String schema : invalidSchema) { + assertEquals(AirbyteProtocolType.UNKNOWN, AirbyteType.fromJsonSchema(Jsons.deserialize(schema))); + } + } + + @Test + public void testChooseOneOf() { + // test ordering + + OneOf o = new OneOf(ImmutableList.of(AirbyteProtocolType.STRING, AirbyteProtocolType.DATE)); + assertEquals(AirbyteProtocolType.DATE, AirbyteTypeUtils.chooseOneOfType(o)); + + final Array a = new Array(AirbyteProtocolType.TIME_WITH_TIMEZONE); + o = new OneOf(ImmutableList.of(AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE, a)); + assertEquals(a, AirbyteTypeUtils.chooseOneOfType(o)); + + final LinkedHashMap properties = new LinkedHashMap<>(); + properties.put("key1", AirbyteProtocolType.UNKNOWN); + properties.put("key2", AirbyteProtocolType.TIME_WITHOUT_TIMEZONE); + final Struct s = new Struct(properties); + o = new OneOf(ImmutableList.of(AirbyteProtocolType.TIMESTAMP_WITHOUT_TIMEZONE, s)); + assertEquals(s, AirbyteTypeUtils.chooseOneOfType(o)); + + // test exclusion + + o = new OneOf(ImmutableList.of(AirbyteProtocolType.BOOLEAN, AirbyteProtocolType.INTEGER)); + assertEquals(AirbyteProtocolType.INTEGER, AirbyteTypeUtils.chooseOneOfType(o)); + + o = new OneOf(ImmutableList.of(AirbyteProtocolType.INTEGER, AirbyteProtocolType.NUMBER, AirbyteProtocolType.DATE)); + assertEquals(AirbyteProtocolType.NUMBER, AirbyteTypeUtils.chooseOneOfType(o)); + + o = new OneOf(ImmutableList.of(AirbyteProtocolType.BOOLEAN, AirbyteProtocolType.NUMBER, AirbyteProtocolType.STRING)); + assertEquals(AirbyteProtocolType.STRING, AirbyteTypeUtils.chooseOneOfType(o)); + } + + @Test + public void testAsColumns() { + OneOf o = new OneOf(List.of( + AirbyteProtocolType.STRING, + new Struct(new LinkedHashMap<>() { + + { + put("foo", AirbyteProtocolType.STRING); + } + + }), + new Array(AirbyteProtocolType.STRING), + // This is bad behavior, but it matches current behavior so we'll test it. + // Ideally, we would recognize that the sub-oneOfs are also objects. + new OneOf(List.of(new Struct(new LinkedHashMap<>()))), + new UnsupportedOneOf(List.of(new Struct(new LinkedHashMap<>()))))); + + LinkedHashMap columns = o.asColumns(); + + assertEquals( + new LinkedHashMap<>() { + + { + put("foo", AirbyteProtocolType.STRING); + } + + }, + columns); + } + + @Test + public void testAsColumnsMultipleObjects() { + OneOf o = new OneOf(List.of( + new Struct(new LinkedHashMap<>()), + new Struct(new LinkedHashMap<>()))); + + // This prooobably should throw an exception, but for the sake of smooth rollout it just logs a + // warning for now. + assertEquals(new LinkedHashMap<>(), o.asColumns()); + } + + @Test + public void testAsColumnsNoObjects() { + OneOf o = new OneOf(List.of( + AirbyteProtocolType.STRING, + new Array(AirbyteProtocolType.STRING), + new UnsupportedOneOf(new ArrayList<>()), + // Similar to testAsColumns(), this is bad behavior. + new OneOf(List.of(new Struct(new LinkedHashMap<>()))), + new UnsupportedOneOf(List.of(new Struct(new LinkedHashMap<>()))))); + + // This prooobably should throw an exception, but for the sake of smooth rollout it just logs a + // warning for now. + assertEquals(new LinkedHashMap<>(), o.asColumns()); + } + +} diff --git a/airbyte-integrations/bases/base-typing-deduping/src/test/java/io/airbyte/integrations/base/destination/typing_deduping/CatalogParserTest.java b/airbyte-integrations/bases/base-typing-deduping/src/test/java/io/airbyte/integrations/base/destination/typing_deduping/CatalogParserTest.java new file mode 100644 index 000000000000..700f3969dfc5 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping/src/test/java/io/airbyte/integrations/base/destination/typing_deduping/CatalogParserTest.java @@ -0,0 +1,143 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.base.destination.typing_deduping; + +import static org.junit.jupiter.api.Assertions.*; +import static org.mockito.Mockito.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.fasterxml.jackson.databind.JsonNode; +import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser.ParsedCatalog; +import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator.ColumnId; +import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator.StreamId; +import io.airbyte.protocol.models.v0.AirbyteStream; +import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog; +import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream; +import java.util.List; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +class CatalogParserTest { + + private SqlGenerator sqlGenerator; + private CatalogParser parser; + + @BeforeEach + public void setup() { + sqlGenerator = mock(SqlGenerator.class); + // noop quoting logic + when(sqlGenerator.buildColumnId(any())).thenAnswer(invocation -> { + String fieldName = invocation.getArgument(0); + return new ColumnId(fieldName, fieldName, fieldName); + }); + when(sqlGenerator.buildStreamId(any(), any(), any())).thenAnswer(invocation -> { + String namespace = invocation.getArgument(0); + String name = invocation.getArgument(1); + String rawNamespace = invocation.getArgument(1); + return new StreamId(namespace, name, rawNamespace, namespace + "_" + name, namespace, name); + }); + + parser = new CatalogParser(sqlGenerator); + } + + /** + * Both these streams want the same raw table name ("a_b_c"). Verify that they don't actually use + * the same raw table. + */ + @Disabled("This feature is not yet supported; see https://github.com/airbytehq/airbyte/issues/27798") + @Test + public void rawNameCollision() { + final ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(List.of( + stream("a", "b_c"), + stream("a_b", "c"))); + + final ParsedCatalog parsedCatalog = parser.parseCatalog(catalog); + + assertNotEquals( + parsedCatalog.streams().get(0).id().rawName(), + parsedCatalog.streams().get(1).id().rawName()); + } + + /** + * Both these streams will write to the same final table name ("foofoo"). Verify that they don't + * actually use the same tablename. + */ + @Test + public void finalNameCollision() { + when(sqlGenerator.buildStreamId(any(), any(), any())).thenAnswer(invocation -> { + String originalNamespace = invocation.getArgument(0); + String originalName = (invocation.getArgument(1)); + String originalRawNamespace = (invocation.getArgument(1)); + + // emulate quoting logic that causes a name collision + String quotedName = originalName.replaceAll("bar", ""); + return new StreamId(originalNamespace, quotedName, originalRawNamespace, originalNamespace + "_" + quotedName, originalNamespace, originalName); + }); + final ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(List.of( + stream("a", "foobarfoo"), + stream("a", "foofoo"))); + + final ParsedCatalog parsedCatalog = parser.parseCatalog(catalog); + + assertNotEquals( + parsedCatalog.streams().get(0).id().finalName(), + parsedCatalog.streams().get(1).id().finalName()); + } + + /** + * The schema contains two fields, which will both end up named "foofoo" after quoting. Verify that + * they don't actually use the same column name. + */ + @Test + public void columnNameCollision() { + when(sqlGenerator.buildColumnId(any())).thenAnswer(invocation -> { + String originalName = invocation.getArgument(0); + + // emulate quoting logic that causes a name collision + String quotedName = originalName.replaceAll("bar", ""); + return new ColumnId(quotedName, originalName, quotedName); + }); + JsonNode schema = Jsons.deserialize(""" + { + "type": "object", + "properties": { + "foobarfoo": {"type": "string"}, + "foofoo": {"type": "string"} + } + } + """); + final ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(List.of(stream("a", "a", schema))); + + final ParsedCatalog parsedCatalog = parser.parseCatalog(catalog); + + assertEquals(2, parsedCatalog.streams().get(0).columns().size()); + } + + private static ConfiguredAirbyteStream stream(String namespace, String name) { + return stream( + namespace, + name, + Jsons.deserialize(""" + { + "type": "object", + "properties": { + "name": {"type": "string"} + } + } + """)); + } + + private static ConfiguredAirbyteStream stream(String namespace, String name, JsonNode schema) { + return new ConfiguredAirbyteStream().withStream( + new AirbyteStream() + .withNamespace(namespace) + .withName(name) + .withJsonSchema(schema)); + } + +} diff --git a/airbyte-integrations/bases/base-typing-deduping/src/test/java/io/airbyte/integrations/base/destination/typing_deduping/TypeAndDedupeOperationValveTest.java b/airbyte-integrations/bases/base-typing-deduping/src/test/java/io/airbyte/integrations/base/destination/typing_deduping/TypeAndDedupeOperationValveTest.java new file mode 100644 index 000000000000..6be4cdd36253 --- /dev/null +++ b/airbyte-integrations/bases/base-typing-deduping/src/test/java/io/airbyte/integrations/base/destination/typing_deduping/TypeAndDedupeOperationValveTest.java @@ -0,0 +1,84 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.base.destination.typing_deduping; + +public class TypeAndDedupeOperationValveTest { + /* + * private static final AirbyteStreamNameNamespacePair STREAM_A = new + * AirbyteStreamNameNamespacePair("a", "a"); private static final AirbyteStreamNameNamespacePair + * STREAM_B = new AirbyteStreamNameNamespacePair("b", "b"); + * + * private static final Supplier ALWAYS_ZERO = () -> 0l; + * + * private Supplier minuteUpdates; + * + * @BeforeEach public void setup() { AtomicLong start = new AtomicLong(0); minuteUpdates = () -> + * start.getAndUpdate(l -> l + (60 * 1000)); } + * + * private void elapseTime(Supplier timing, int iterations) { IntStream.range(0, + * iterations).forEach(__ -> { timing.get(); }); } + * + * @Test public void testAddStream() { final var valve = new + * TypeAndDedupeOperationValve(ALWAYS_ZERO); valve.addStream(STREAM_A); Assertions.assertEquals(1l, + * valve.getRecordCount(STREAM_A)); Assertions.assertEquals(1000 * 60 * 2, + * valve.getIncrementInterval(STREAM_A)); + * Assertions.assertFalse(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A)); + * Assertions.assertEquals(valve.get(STREAM_A), 0l); } + * + * @Test public void testReadyToTypeAndDedupe() { final var valve = new + * TypeAndDedupeOperationValve(minuteUpdates); // method call increments time + * valve.addStream(STREAM_A); IntStream.range(0, 98).forEach(__ -> + * Assertions.assertFalse(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A))); + * elapseTime(minuteUpdates, 1); // method call increments time valve.addStream(STREAM_B); // method + * call increments time + * Assertions.assertTrue(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A)); + * IntStream.range(0, 98).forEach(__ -> + * Assertions.assertFalse(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_B))); + * elapseTime(minuteUpdates, 1); + * Assertions.assertTrue(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_B)); + * valve.updateTimeAndIncreaseInterval(STREAM_A); Assertions.assertEquals(1000 * 60 * 5, + * valve.getIncrementInterval(STREAM_A)); IntStream.range(0, 99).forEach(__ -> + * Assertions.assertFalse(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A))); + * Assertions.assertEquals(199, valve.getRecordCount(STREAM_A)); // method call increments time // + * This puts it at 200 records, which should then check to see if enough time has passed // but only + * one minute has passed + * Assertions.assertFalse(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A)); + * elapseTime(minuteUpdates, 5); IntStream.range(0, 99).forEach(__ -> + * Assertions.assertFalse(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A))); + * Assertions.assertEquals(299, valve.getRecordCount(STREAM_A)); // More than enough time has passed + * now and this will be the 300th record + * Assertions.assertTrue(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A)); } + * + * @Test public void testIncrementInterval() { final var valve = new + * TypeAndDedupeOperationValve(ALWAYS_ZERO); valve.addStream(STREAM_A); IntStream.rangeClosed(1, + * 3).forEach(i -> { final var index = valve.incrementInterval(STREAM_A); Assertions.assertEquals(i, + * index); }); Assertions.assertEquals(3, valve.incrementInterval(STREAM_A)); // Twice to be sure + * Assertions.assertEquals(3, valve.incrementInterval(STREAM_A)); } + * + * @Test public void testUpdateTimeAndIncreaseInterval() { final var valve = new + * TypeAndDedupeOperationValve(minuteUpdates); valve.addStream(STREAM_A); // 2 minutes + * IntStream.range(0, 98).forEach(__ -> + * Assertions.assertFalse(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A))); + * elapseTime(minuteUpdates, 2); + * Assertions.assertTrue(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A)); + * valve.updateTimeAndIncreaseInterval(STREAM_A); // 5 minutes IntStream.range(0, 99).forEach(__ -> + * Assertions.assertFalse(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A))); + * elapseTime(minuteUpdates, 5); + * Assertions.assertTrue(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A)); + * valve.updateTimeAndIncreaseInterval(STREAM_A); // 10 minutes IntStream.range(0, 99).forEach(__ -> + * Assertions.assertFalse(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A))); + * elapseTime(minuteUpdates, 10); + * Assertions.assertTrue(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A)); + * valve.updateTimeAndIncreaseInterval(STREAM_A); // 15 minutes IntStream.range(0, 99).forEach(__ -> + * Assertions.assertFalse(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A))); + * elapseTime(minuteUpdates, 15); + * Assertions.assertTrue(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A)); + * valve.updateTimeAndIncreaseInterval(STREAM_A); // 15 minutes again IntStream.range(0, + * 99).forEach(__ -> + * Assertions.assertFalse(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A))); + * elapseTime(minuteUpdates, 15); + * Assertions.assertTrue(valve.readyToTypeAndDedupeWithAdditionalRecord(STREAM_A)); } + */ +} diff --git a/airbyte-integrations/bases/standard-destination-test/src/main/java/io/airbyte/integrations/standardtest/destination/DestinationAcceptanceTest.java b/airbyte-integrations/bases/standard-destination-test/src/main/java/io/airbyte/integrations/standardtest/destination/DestinationAcceptanceTest.java index 6e2857b30ed1..0263d7c2b610 100644 --- a/airbyte-integrations/bases/standard-destination-test/src/main/java/io/airbyte/integrations/standardtest/destination/DestinationAcceptanceTest.java +++ b/airbyte-integrations/bases/standard-destination-test/src/main/java/io/airbyte/integrations/standardtest/destination/DestinationAcceptanceTest.java @@ -1081,11 +1081,11 @@ void testSyncWriteSameTableNameDifferentNamespace() throws Exception { Jsons.deserialize( MoreResources.readResource(DataArgumentsProvider.EXCHANGE_RATE_CONFIG.getCatalogFileVersion(getProtocolVersion())), AirbyteCatalog.class); - final var namespace1 = "sourcenamespace"; + final var namespace1 = Strings.addRandomSuffix("sourcenamespace", "_", 8); catalog.getStreams().forEach(stream -> stream.setNamespace(namespace1)); final var diffNamespaceStreams = new ArrayList(); - final var namespace2 = "diff_source_namespace"; + final var namespace2 = Strings.addRandomSuffix("diff_sourcenamespace", "_", 8);; final var mapper = MoreMappers.initMapper(); for (final AirbyteStream stream : catalog.getStreams()) { final var clonedStream = mapper.readValue(mapper.writeValueAsString(stream), diff --git a/airbyte-integrations/connectors/destination-bigquery-denormalized/Dockerfile b/airbyte-integrations/connectors/destination-bigquery-denormalized/Dockerfile index 04c00f17ac90..027798aee23d 100644 --- a/airbyte-integrations/connectors/destination-bigquery-denormalized/Dockerfile +++ b/airbyte-integrations/connectors/destination-bigquery-denormalized/Dockerfile @@ -24,5 +24,5 @@ ENV APPLICATION destination-bigquery-denormalized COPY --from=build /airbyte /airbyte -LABEL io.airbyte.version=1.5.1 +LABEL io.airbyte.version=1.5.2 LABEL io.airbyte.name=airbyte/destination-bigquery-denormalized diff --git a/airbyte-integrations/connectors/destination-bigquery-denormalized/build.gradle b/airbyte-integrations/connectors/destination-bigquery-denormalized/build.gradle index 2516657a500d..bab42c9c7c4a 100644 --- a/airbyte-integrations/connectors/destination-bigquery-denormalized/build.gradle +++ b/airbyte-integrations/connectors/destination-bigquery-denormalized/build.gradle @@ -26,7 +26,6 @@ dependencies { integrationTestJavaImplementation project(':airbyte-integrations:bases:standard-destination-test') integrationTestJavaImplementation project(':airbyte-integrations:connectors:destination-bigquery-denormalized') - integrationTestJavaImplementation files(project(':airbyte-integrations:bases:base-normalization').airbyteDocker.outputs) integrationTestJavaImplementation project(':airbyte-db:db-lib') implementation files(project(':airbyte-integrations:bases:base-java').airbyteDocker.outputs) diff --git a/airbyte-integrations/connectors/destination-bigquery-denormalized/metadata.yaml b/airbyte-integrations/connectors/destination-bigquery-denormalized/metadata.yaml index f8aa8cc7270e..ca0f0ebb5992 100644 --- a/airbyte-integrations/connectors/destination-bigquery-denormalized/metadata.yaml +++ b/airbyte-integrations/connectors/destination-bigquery-denormalized/metadata.yaml @@ -2,7 +2,7 @@ data: connectorSubtype: database connectorType: destination definitionId: 079d5540-f236-4294-ba7c-ade8fd918496 - dockerImageTag: 1.5.1 + dockerImageTag: 1.5.2 dockerRepository: airbyte/destination-bigquery-denormalized githubIssueLabel: destination-bigquery-denormalized icon: bigquery.svg @@ -10,10 +10,8 @@ data: name: BigQuery (denormalized typed struct) registries: cloud: - dockerImageTag: 1.5.1 enabled: true oss: - dockerImageTag: 1.5.1 enabled: true releaseStage: beta resourceRequirements: diff --git a/airbyte-integrations/connectors/destination-bigquery/Dockerfile b/airbyte-integrations/connectors/destination-bigquery/Dockerfile index 0124fce3b801..be211bd93238 100644 --- a/airbyte-integrations/connectors/destination-bigquery/Dockerfile +++ b/airbyte-integrations/connectors/destination-bigquery/Dockerfile @@ -25,7 +25,7 @@ RUN yum install -y python3 python3-devel jq sshpass git && yum clean all && \ # Luckily, none of normalization's files conflict with destination-bigquery's files :) # We don't enforce that in any way, but hopefully we're only living in this state for a short time. -COPY --from=airbyte/normalization:dev /airbyte /airbyte +COPY --from=airbyte/normalization:0.4.3 /airbyte /airbyte # Install python dependencies WORKDIR /airbyte/base_python_structs RUN pip3 install . @@ -44,7 +44,7 @@ ENV AIRBYTE_NORMALIZATION_INTEGRATION bigquery COPY --from=build /airbyte /airbyte -LABEL io.airbyte.version=1.5.1 +LABEL io.airbyte.version=1.5.2 LABEL io.airbyte.name=airbyte/destination-bigquery ENV AIRBYTE_ENTRYPOINT "/airbyte/run_with_normalization.sh" diff --git a/airbyte-integrations/connectors/destination-bigquery/build.gradle b/airbyte-integrations/connectors/destination-bigquery/build.gradle index 49cef6bab508..19c645dd13a7 100644 --- a/airbyte-integrations/connectors/destination-bigquery/build.gradle +++ b/airbyte-integrations/connectors/destination-bigquery/build.gradle @@ -10,9 +10,10 @@ application { } dependencies { - implementation 'com.google.cloud:google-cloud-bigquery:1.122.2' + implementation 'com.google.cloud:google-cloud-bigquery:2.27.0' implementation 'org.apache.commons:commons-lang3:3.11' implementation 'org.apache.commons:commons-csv:1.4' + implementation 'org.apache.commons:commons-text:1.10.0' implementation group: 'org.apache.parquet', name: 'parquet-avro', version: '1.12.0' implementation group: 'com.google.cloud', name: 'google-cloud-storage', version: '2.4.5' @@ -22,10 +23,12 @@ dependencies { implementation project(':airbyte-integrations:bases:base-java') implementation libs.airbyte.protocol implementation project(':airbyte-integrations:bases:base-java-s3') + implementation project(':airbyte-integrations:bases:base-typing-deduping') implementation project(':airbyte-integrations:connectors:destination-gcs') implementation ('com.github.airbytehq:json-avro-converter:1.1.0') { exclude group: 'ch.qos.logback', module: 'logback-classic'} testImplementation project(':airbyte-integrations:bases:standard-destination-test') + testImplementation project(':airbyte-integrations:bases:base-typing-deduping-test') integrationTestJavaImplementation project(':airbyte-integrations:bases:standard-destination-test') integrationTestJavaImplementation project(':airbyte-integrations:connectors:destination-bigquery') @@ -34,12 +37,6 @@ dependencies { implementation files(project(':airbyte-integrations:bases:base-java').airbyteDocker.outputs) } -tasks.named("airbyteDocker") { - // this is really inefficent (because base-normalization:airbyteDocker builds 9 docker images) - // but it's also just simple to implement - dependsOn project(':airbyte-integrations:bases:base-normalization').airbyteDocker -} - configurations.all { resolutionStrategy { // at time of writing: deps.toml declares google-cloud-storage 2.17.2 diff --git a/airbyte-integrations/connectors/destination-bigquery/metadata.yaml b/airbyte-integrations/connectors/destination-bigquery/metadata.yaml index be45a674ce36..4af605a2d63d 100644 --- a/airbyte-integrations/connectors/destination-bigquery/metadata.yaml +++ b/airbyte-integrations/connectors/destination-bigquery/metadata.yaml @@ -2,7 +2,7 @@ data: connectorSubtype: database connectorType: destination definitionId: 22f6c74f-5699-40ff-833c-4a879ea40133 - dockerImageTag: 1.5.1 + dockerImageTag: 1.5.2 dockerRepository: airbyte/destination-bigquery githubIssueLabel: destination-bigquery icon: bigquery.svg diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryAvroSerializedBuffer.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryAvroSerializedBuffer.java index 7c966c3c0e05..9f68f48e70fd 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryAvroSerializedBuffer.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryAvroSerializedBuffer.java @@ -19,7 +19,6 @@ import java.util.function.Function; import org.apache.avro.Schema; import org.apache.avro.file.CodecFactory; -import org.apache.commons.lang3.StringUtils; /** * This class differs from {@link AvroSerializedBuffer} in that 1) the Avro schema can be customized @@ -52,7 +51,7 @@ public static BufferCreateFunction createBufferFunction(final S3AvroFormatConfig return (pair, catalog) -> { final AirbyteStream stream = catalog.getStreams() .stream() - .filter(s -> s.getStream().getName().equals(pair.getName()) && StringUtils.equals(s.getStream().getNamespace(), pair.getNamespace())) + .filter(s -> s.getStream().getName().equals(pair.getName())) .findFirst() .orElseThrow(() -> new RuntimeException(String.format("No such stream %s.%s", pair.getNamespace(), pair.getName()))) .getStream(); diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryDestination.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryDestination.java index 58519e1ad4ce..b4e7ba499f9b 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryDestination.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryDestination.java @@ -16,17 +16,24 @@ import com.google.cloud.storage.StorageOptions; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Charsets; +import com.google.common.base.Strings; import io.airbyte.commons.exceptions.ConfigErrorException; import io.airbyte.commons.json.Jsons; import io.airbyte.integrations.BaseConnector; import io.airbyte.integrations.base.AirbyteMessageConsumer; import io.airbyte.integrations.base.Destination; import io.airbyte.integrations.base.IntegrationRunner; +import io.airbyte.integrations.base.TypingAndDedupingFlag; +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser; +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser.ParsedCatalog; +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser.StreamConfig; import io.airbyte.integrations.destination.StandardNameTransformer; import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter; import io.airbyte.integrations.destination.bigquery.formatter.DefaultBigQueryRecordFormatter; import io.airbyte.integrations.destination.bigquery.formatter.GcsAvroBigQueryRecordFormatter; import io.airbyte.integrations.destination.bigquery.formatter.GcsCsvBigQueryRecordFormatter; +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler; +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator; import io.airbyte.integrations.destination.bigquery.uploader.AbstractBigQueryUploader; import io.airbyte.integrations.destination.bigquery.uploader.BigQueryUploaderFactory; import io.airbyte.integrations.destination.bigquery.uploader.UploaderType; @@ -66,6 +73,7 @@ public class BigQueryDestination extends BaseConnector implements Destination { + public static final String RAW_NAMESPACE_OVERRIDE = "raw_data_dataset"; private static final Logger LOGGER = LoggerFactory.getLogger(BigQueryDestination.class); private static final List REQUIRED_PERMISSIONS = List.of( "storage.multipartUploads.abort", @@ -160,7 +168,7 @@ private AirbyteConnectionStatus checkGcsPermission(final JsonNode config) { } } - protected BigQuery getBigQuery(final JsonNode config) { + public static BigQuery getBigQuery(final JsonNode config) { final String projectId = config.get(BigQueryConsts.CONFIG_PROJECT_ID).asText(); try { @@ -199,45 +207,72 @@ public static GoogleCredentials getServiceAccountCredentials(final JsonNode conf * @param config - integration-specific configuration object as json. e.g. { "username": "airbyte", * "password": "super secure" } * @param catalog - schema of the incoming messages. - * @param outputRecordCollector - * @return - * @throws IOException */ @Override public AirbyteMessageConsumer getConsumer(final JsonNode config, final ConfiguredAirbyteCatalog catalog, final Consumer outputRecordCollector) - throws IOException { + throws IOException, InterruptedException { + // Set the default namespace on streams with null namespace. This means we don't need to repeat this + // logic in the rest of the connector. + // (record messages still need to handle null namespaces though, which currently happens in e.g. + // BigQueryRecordConsumer#acceptTracked) + // This probably should be shared logic amongst destinations eventually. + for (final ConfiguredAirbyteStream stream : catalog.getStreams()) { + if (StringUtils.isEmpty(stream.getStream().getNamespace())) { + stream.getStream().withNamespace(BigQueryUtils.getDatasetId(config)); + } + } + + final BigQuerySqlGenerator sqlGenerator = new BigQuerySqlGenerator(); + final CatalogParser catalogParser; + if (config.hasNonNull(RAW_NAMESPACE_OVERRIDE) && !Strings.isNullOrEmpty(config.get(RAW_NAMESPACE_OVERRIDE).asText())) { + catalogParser = new CatalogParser(sqlGenerator, config.get(RAW_NAMESPACE_OVERRIDE).asText()); + } else { + catalogParser = new CatalogParser(sqlGenerator); + } + ParsedCatalog parsedCatalog = catalogParser.parseCatalog(catalog); + final UploadingMethod uploadingMethod = BigQueryUtils.getLoadingMethod(config); if (uploadingMethod == UploadingMethod.STANDARD) { LOGGER.warn("The \"standard\" upload mode is not performant, and is not recommended for production. " + "Please use the GCS upload mode if you are syncing a large amount of data."); - return getStandardRecordConsumer(config, catalog, outputRecordCollector); + return getStandardRecordConsumer(config, catalog, parsedCatalog, outputRecordCollector, sqlGenerator); } else { - return getGcsRecordConsumer(config, catalog, outputRecordCollector); + return getGcsRecordConsumer(config, catalog, parsedCatalog, outputRecordCollector, sqlGenerator); } } - protected Map> getUploaderMap(final JsonNode config, - final ConfiguredAirbyteCatalog catalog) + protected Map> getUploaderMap( + final BigQuery bigquery, + final JsonNode config, + final ConfiguredAirbyteCatalog catalog, + final ParsedCatalog parsedCatalog, + final boolean use1s1t) throws IOException { - final BigQuery bigquery = getBigQuery(config); - final Map> uploaderMap = new HashMap<>(); for (final ConfiguredAirbyteStream configStream : catalog.getStreams()) { final AirbyteStream stream = configStream.getStream(); - if (StringUtils.isEmpty(stream.getNamespace())) { - stream.setNamespace(BigQueryUtils.getDatasetId(config)); - } + StreamConfig parsedStream = parsedCatalog.getStream(stream.getNamespace(), stream.getName()); + final String streamName = stream.getName(); + String targetTableName; + if (use1s1t) { + targetTableName = parsedStream.id().rawName(); + } else { + targetTableName = getTargetTableName(streamName); + } + final UploaderConfig uploaderConfig = UploaderConfig .builder() .bigQuery(bigquery) .configStream(configStream) + .parsedStream(parsedStream) .config(config) .formatterMap(getFormatterMap(stream.getJsonSchema())) .tmpTableName(namingResolver.getTmpTableName(streamName)) - .targetTableName(getTargetTableName(streamName)) + .targetTableName(targetTableName) + // This refers to whether this is BQ denormalized or not .isDefaultAirbyteTmpSchema(isDefaultAirbyteTmpTableSchema()) .build(); @@ -278,16 +313,35 @@ protected String getTargetTableName(final String streamName) { private AirbyteMessageConsumer getStandardRecordConsumer(final JsonNode config, final ConfiguredAirbyteCatalog catalog, - final Consumer outputRecordCollector) + final ParsedCatalog parsedCatalog, + final Consumer outputRecordCollector, + BigQuerySqlGenerator sqlGenerator) throws IOException { - final Map> writeConfigs = getUploaderMap(config, catalog); - return new BigQueryRecordConsumer(writeConfigs, outputRecordCollector, BigQueryUtils.getDatasetId(config)); + final BigQuery bigquery = getBigQuery(config); + + final Map> writeConfigs = getUploaderMap( + bigquery, + config, + catalog, + parsedCatalog, + TypingAndDedupingFlag.isDestinationV2()); + + return new BigQueryRecordConsumer( + bigquery, + writeConfigs, + outputRecordCollector, + BigQueryUtils.getDatasetId(config), + sqlGenerator, + new BigQueryDestinationHandler(bigquery), + parsedCatalog); } public AirbyteMessageConsumer getGcsRecordConsumer(final JsonNode config, final ConfiguredAirbyteCatalog catalog, - final Consumer outputRecordCollector) { - + final ParsedCatalog parsedCatalog, + final Consumer outputRecordCollector, + final BigQuerySqlGenerator sqlGenerator) + throws InterruptedException { final StandardNameTransformer gcsNameTransformer = new GcsNameTransformer(); final BigQuery bigQuery = getBigQuery(config); final GcsDestinationConfig gcsConfig = BigQueryUtils.getGcsAvroDestinationConfig(config); @@ -323,6 +377,7 @@ public AirbyteMessageConsumer getGcsRecordConsumer(final JsonNode config, () -> new FileBuffer(S3AvroFormatConfig.DEFAULT_SUFFIX, numberOfFileBuffers)); LOGGER.info("Creating BigQuery staging message consumer with staging ID {} at {}", stagingId, syncDatetime); + return new BigQueryStagingConsumerFactory().create( config, catalog, @@ -331,7 +386,11 @@ public AirbyteMessageConsumer getGcsRecordConsumer(final JsonNode config, onCreateBuffer, recordFormatterCreator, namingResolver::getTmpTableName, - getTargetTableNameTransformer(namingResolver)); + getTargetTableNameTransformer(namingResolver), + sqlGenerator, + new BigQueryDestinationHandler(bigQuery), + parsedCatalog, + BigQueryUtils.getDatasetId(config)); } protected BiFunction getAvroSchemaCreator() { @@ -349,7 +408,7 @@ protected Function getTargetTableNameTransformer(final BigQueryS /** * Retrieves user configured file buffer amount so as long it doesn't exceed the maximum number of * file buffers and sets the minimum number to the default - * + *

* NOTE: If Out Of Memory Exceptions (OOME) occur, this can be a likely cause as this hard limit has * not been thoroughly load tested across all instance sizes * diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryRecordConsumer.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryRecordConsumer.java index aa3069a06b04..1e1703e36484 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryRecordConsumer.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryRecordConsumer.java @@ -4,16 +4,31 @@ package io.airbyte.integrations.destination.bigquery; +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.TableDefinition; +import com.google.cloud.bigquery.TableId; import io.airbyte.commons.string.Strings; import io.airbyte.integrations.base.AirbyteMessageConsumer; import io.airbyte.integrations.base.FailureTrackingAirbyteMessageConsumer; +import io.airbyte.integrations.base.TypingAndDedupingFlag; +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser.ParsedCatalog; +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser.StreamConfig; +import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator.StreamId; +import io.airbyte.integrations.destination.bigquery.formatter.DefaultBigQueryRecordFormatter; +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler; +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator; +import io.airbyte.integrations.base.destination.typing_deduping.TypeAndDedupeOperationValve; import io.airbyte.integrations.destination.bigquery.uploader.AbstractBigQueryUploader; import io.airbyte.protocol.models.v0.AirbyteMessage; import io.airbyte.protocol.models.v0.AirbyteMessage.Type; import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair; +import io.airbyte.protocol.models.v0.DestinationSyncMode; +import java.math.BigInteger; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.function.Consumer; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -24,24 +39,98 @@ */ public class BigQueryRecordConsumer extends FailureTrackingAirbyteMessageConsumer implements AirbyteMessageConsumer { + public static final String OVERWRITE_TABLE_SUFFIX = "_airbyte_tmp"; + private static final Logger LOGGER = LoggerFactory.getLogger(BigQueryRecordConsumer.class); + private final BigQuery bigquery; private final Map> uploaderMap; private final Consumer outputRecordCollector; - private final String datasetId; + private final String defaultDatasetId; + private final BigQuerySqlGenerator sqlGenerator; + private final BigQueryDestinationHandler destinationHandler; private AirbyteMessage lastStateMessage = null; - public BigQueryRecordConsumer(final Map> uploaderMap, + private final TypeAndDedupeOperationValve streamTDValve = new TypeAndDedupeOperationValve(); + private final ParsedCatalog catalog; + private final boolean use1s1t; + private final Map overwriteStreamsWithTmpTable; + + public BigQueryRecordConsumer(final BigQuery bigquery, + final Map> uploaderMap, final Consumer outputRecordCollector, - final String datasetId) { + final String defaultDatasetId, + final BigQuerySqlGenerator sqlGenerator, + final BigQueryDestinationHandler destinationHandler, + final ParsedCatalog catalog) { + this.bigquery = bigquery; this.uploaderMap = uploaderMap; this.outputRecordCollector = outputRecordCollector; - this.datasetId = datasetId; + this.defaultDatasetId = defaultDatasetId; + this.sqlGenerator = sqlGenerator; + this.destinationHandler = destinationHandler; + this.catalog = catalog; + this.overwriteStreamsWithTmpTable = new HashMap<>(); + + this.use1s1t = TypingAndDedupingFlag.isDestinationV2(); + + LOGGER.info("Got parsed catalog {}", catalog); + LOGGER.info("Got canonical stream IDs {}", uploaderMap.keySet()); } @Override - protected void startTracked() { + protected void startTracked() throws InterruptedException { // todo (cgardens) - move contents of #write into this method. + + if (use1s1t) { + // TODO extract common logic with GCS record consumer + extract into a higher level class + // For each stream, make sure that its corresponding final table exists. + for (final StreamConfig stream : catalog.streams()) { + final Optional existingTable = destinationHandler.findExistingTable(stream.id()); + if (existingTable.isEmpty()) { + destinationHandler.execute(sqlGenerator.createTable(stream, "")); + if (stream.destinationSyncMode() == DestinationSyncMode.OVERWRITE) { + // We're creating this table for the first time. Write directly into it. + overwriteStreamsWithTmpTable.put(stream.id(), ""); + } + } else { + destinationHandler.execute(sqlGenerator.alterTable(stream, existingTable.get())); + if (stream.destinationSyncMode() == DestinationSyncMode.OVERWRITE) { + final BigInteger rowsInFinalTable = bigquery.getTable(TableId.of(stream.id().finalNamespace(), stream.id().finalName())).getNumRows(); + if (new BigInteger("0").equals(rowsInFinalTable)) { + // The table already exists but is empty. We'll load data incrementally. + // (this might be because the user ran a reset, which creates an empty table) + overwriteStreamsWithTmpTable.put(stream.id(), ""); + } else { + // We're working with an existing table. Write into a tmp table. We'll overwrite the table at the + // end of the sync. + overwriteStreamsWithTmpTable.put(stream.id(), OVERWRITE_TABLE_SUFFIX); + } + } + } + } + + // For streams in overwrite mode, truncate the raw table and create a tmp table. + // non-1s1t syncs actually overwrite the raw table at the end of the sync, so we only do this in + // 1s1t mode. + for (final StreamConfig stream : catalog.streams()) { + LOGGER.info("Stream {} has sync mode {}", stream.id(), stream.destinationSyncMode()); + final String suffix = overwriteStreamsWithTmpTable.get(stream.id()); + if (stream.destinationSyncMode() == DestinationSyncMode.OVERWRITE && suffix != null && !suffix.isEmpty()) { + // drop+recreate the raw table + final TableId rawTableId = TableId.of(stream.id().rawNamespace(), stream.id().rawName()); + bigquery.delete(rawTableId); + BigQueryUtils.createPartitionedTableIfNotExists(bigquery, rawTableId, DefaultBigQueryRecordFormatter.SCHEMA_V2); + + // create the tmp final table + destinationHandler.execute(sqlGenerator.createTable(stream, suffix)); + } + } + + uploaderMap.forEach((streamId, uploader) -> { + uploader.createRawTable(); + }); + } } /** @@ -54,13 +143,13 @@ protected void startTracked() { * @param message {@link AirbyteMessage} to be processed */ @Override - public void acceptTracked(final AirbyteMessage message) { + public void acceptTracked(final AirbyteMessage message) throws InterruptedException { if (message.getType() == Type.STATE) { lastStateMessage = message; outputRecordCollector.accept(message); } else if (message.getType() == Type.RECORD) { if (StringUtils.isEmpty(message.getRecord().getNamespace())) { - message.getRecord().setNamespace(datasetId); + message.getRecord().setNamespace(defaultDatasetId); } processRecord(message); } else { @@ -74,18 +163,38 @@ public void acceptTracked(final AirbyteMessage message) { * * @param message record to be written */ - private void processRecord(final AirbyteMessage message) { - final var pair = AirbyteStreamNameNamespacePair.fromRecordMessage(message.getRecord()); - uploaderMap.get(pair).upload(message); + private void processRecord(final AirbyteMessage message) throws InterruptedException { + final var streamId = AirbyteStreamNameNamespacePair.fromRecordMessage(message.getRecord()); + uploaderMap.get(streamId).upload(message); + if (!streamTDValve.containsKey(streamId)) { + streamTDValve.addStream(streamId); + } else if (streamTDValve.readyToTypeAndDedupeWithAdditionalRecord(streamId)) { + doTypingAndDeduping(catalog.getStream(streamId.getNamespace(), streamId.getName())); + streamTDValve.updateTimeAndIncreaseInterval(streamId); + } } @Override public void close(final boolean hasFailed) { LOGGER.info("Started closing all connections"); final List exceptionsThrown = new ArrayList<>(); - uploaderMap.values().forEach(uploader -> { + uploaderMap.forEach((streamId, uploader) -> { try { uploader.close(hasFailed, outputRecordCollector, lastStateMessage); + if (use1s1t) { + LOGGER.info("Attempting typing and deduping for {}", streamId); + final StreamConfig streamConfig = catalog.getStream(streamId.getNamespace(), streamId.getName()); + doTypingAndDeduping(streamConfig); + if (streamConfig.destinationSyncMode() == DestinationSyncMode.OVERWRITE) { + LOGGER.info("Overwriting final table with tmp table"); + // We're at the end of the sync. Move the tmp table to the final table. + final Optional overwriteFinalTable = + sqlGenerator.overwriteFinalTable(overwriteStreamsWithTmpTable.get(streamConfig.id()), streamConfig); + if (overwriteFinalTable.isPresent()) { + destinationHandler.execute(overwriteFinalTable.get()); + } + } + } } catch (final Exception e) { exceptionsThrown.add(e); LOGGER.error("Exception while closing uploader {}", uploader, e); @@ -96,4 +205,13 @@ public void close(final boolean hasFailed) { } } + private void doTypingAndDeduping(final StreamConfig stream) throws InterruptedException { + if (use1s1t) { + final String suffix; + suffix = overwriteStreamsWithTmpTable.getOrDefault(stream.id(), ""); + final String sql = sqlGenerator.updateTable(suffix, stream); + destinationHandler.execute(sql); + } + } + } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryStagingConsumerFactory.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryStagingConsumerFactory.java index 3e6b96042378..ecb24c4962a8 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryStagingConsumerFactory.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryStagingConsumerFactory.java @@ -4,12 +4,23 @@ package io.airbyte.integrations.destination.bigquery; +import static io.airbyte.integrations.base.JavaBaseConstants.AIRBYTE_NAMESPACE_SCHEMA; +import static io.airbyte.integrations.destination.bigquery.BigQueryRecordConsumer.OVERWRITE_TABLE_SUFFIX; + import com.fasterxml.jackson.databind.JsonNode; +import com.google.cloud.bigquery.TableDefinition; import com.google.common.base.Functions; import com.google.common.base.Preconditions; +import io.airbyte.commons.functional.CheckedConsumer; import io.airbyte.commons.json.Jsons; import io.airbyte.integrations.base.AirbyteMessageConsumer; +import io.airbyte.integrations.base.TypingAndDedupingFlag; +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser.ParsedCatalog; +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser.StreamConfig; +import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator; 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.buffered_stream_consumer.BufferedStreamConsumer; import io.airbyte.integrations.destination.buffered_stream_consumer.OnCloseFunction; import io.airbyte.integrations.destination.buffered_stream_consumer.OnStartFunction; @@ -21,8 +32,11 @@ import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair; import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog; import io.airbyte.protocol.models.v0.DestinationSyncMode; +import java.math.BigInteger; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Collectors; @@ -46,28 +60,69 @@ public AirbyteMessageConsumer create(final JsonNode config, final BufferCreateFunction onCreateBuffer, final Function recordFormatterCreator, final Function tmpTableNameTransformer, - final Function targetTableNameTransformer) { + final Function targetTableNameTransformer, + final BigQuerySqlGenerator sqlGenerator, + final BigQueryDestinationHandler destinationHandler, + final ParsedCatalog parsedCatalog, + final String defaultNamespace) + throws InterruptedException { + boolean use1s1t = TypingAndDedupingFlag.isDestinationV2(); final Map writeConfigs = createWriteConfigs( config, catalog, + parsedCatalog, recordFormatterCreator, tmpTableNameTransformer, targetTableNameTransformer); + final var overwriteStreamsWithTmpTable = createFinalTables(use1s1t, parsedCatalog, destinationHandler, sqlGenerator); + CheckedConsumer typeAndDedupeStreamFunction = typingAndDedupingStreamConsumer( + sqlGenerator, + destinationHandler, + parsedCatalog, + use1s1t, + overwriteStreamsWithTmpTable); + + CheckedConsumer replaceFinalTableConsumer = + getReplaceFinalTableConsumer( + use1s1t, + sqlGenerator, + destinationHandler, + overwriteStreamsWithTmpTable, + parsedCatalog); + return new BufferedStreamConsumer( outputRecordCollector, onStartFunction(bigQueryGcsOperations, writeConfigs), new SerializedBufferingStrategy( onCreateBuffer, catalog, - flushBufferFunction(bigQueryGcsOperations, writeConfigs, catalog)), - onCloseFunction(bigQueryGcsOperations, writeConfigs), + flushBufferFunction(bigQueryGcsOperations, writeConfigs, catalog, typeAndDedupeStreamFunction)), + onCloseFunction(bigQueryGcsOperations, writeConfigs, replaceFinalTableConsumer), catalog, - json -> true); + json -> true, + defaultNamespace); + } + + private CheckedConsumer typingAndDedupingStreamConsumer(final BigQuerySqlGenerator sqlGenerator, + final BigQueryDestinationHandler destinationHandler, + final ParsedCatalog parsedCatalog, + final boolean use1s1t, + final Map overwriteStreamsWithTmpTable) { + return (streamId) -> { + if (use1s1t) { + final var streamConfig = parsedCatalog.getStream(streamId.getNamespace(), streamId.getName()); + String suffix; + suffix = overwriteStreamsWithTmpTable.getOrDefault(streamConfig.id(), ""); + final String sql = sqlGenerator.updateTable(suffix, streamConfig); + destinationHandler.execute(sql); + } + }; } private Map createWriteConfigs(final JsonNode config, final ConfiguredAirbyteCatalog catalog, + final ParsedCatalog parsedCatalog, final Function recordFormatterCreator, final Function tmpTableNameTransformer, final Function targetTableNameTransformer) { @@ -76,16 +131,21 @@ private Map createWriteConf Preconditions.checkNotNull(configuredStream.getDestinationSyncMode(), "Undefined destination sync mode"); final AirbyteStream stream = configuredStream.getStream(); + StreamConfig streamConfig = parsedCatalog.getStream(stream.getNamespace(), stream.getName()); final String streamName = stream.getName(); final BigQueryRecordFormatter recordFormatter = recordFormatterCreator.apply(stream.getJsonSchema()); + final var internalTableNamespace = TypingAndDedupingFlag.isDestinationV2() ? streamConfig.id().rawNamespace() : BigQueryUtils.sanitizeDatasetId(stream.getNamespace()); + final var targetTableName = + TypingAndDedupingFlag.isDestinationV2() ? streamConfig.id().rawName() : targetTableNameTransformer.apply(streamName); + final BigQueryWriteConfig writeConfig = new BigQueryWriteConfig( streamName, stream.getNamespace(), - BigQueryUtils.getSchema(config, configuredStream), + internalTableNamespace, BigQueryUtils.getDatasetLocation(config), tmpTableNameTransformer.apply(streamName), - targetTableNameTransformer.apply(streamName), + targetTableName, recordFormatter.getBigQuerySchema(), configuredStream.getDestinationSyncMode()); @@ -107,7 +167,6 @@ private Map createWriteConf * * @param bigQueryGcsOperations collection of Google Cloud Storage Operations * @param writeConfigs configuration settings used to describe how to write data and where it exists - * @return */ private OnStartFunction onStartFunction(final BigQueryStagingOperations bigQueryGcsOperations, final Map writeConfigs) { @@ -116,21 +175,65 @@ private OnStartFunction onStartFunction(final BigQueryStagingOperations bigQuery for (final BigQueryWriteConfig writeConfig : writeConfigs.values()) { LOGGER.info("Preparing staging are in destination for schema: {}, stream: {}, target table: {}, stage: {}", writeConfig.tableSchema(), writeConfig.streamName(), writeConfig.targetTableId(), writeConfig.streamName()); - final String datasetId = writeConfig.datasetId(); - bigQueryGcsOperations.createSchemaIfNotExists(datasetId, writeConfig.datasetLocation()); + // In Destinations V2, we will always use the 'airbyte' schema/namespace for raw tables + final String rawDatasetId = TypingAndDedupingFlag.isDestinationV2() ? AIRBYTE_NAMESPACE_SCHEMA : writeConfig.datasetId(); + // Regardless, ensure the schema the customer wants to write to exists + bigQueryGcsOperations.createSchemaIfNotExists(writeConfig.datasetId(), writeConfig.datasetLocation()); + // Schema used for raw and airbyte internal tables + bigQueryGcsOperations.createSchemaIfNotExists(rawDatasetId, writeConfig.datasetLocation()); + // Customer's destination schema // With checkpointing, we will be creating the target table earlier in the setup such that // the data can be immediately loaded from the staging area bigQueryGcsOperations.createTableIfNotExists(writeConfig.targetTableId(), writeConfig.tableSchema()); - bigQueryGcsOperations.createStageIfNotExists(datasetId, writeConfig.streamName()); + bigQueryGcsOperations.createStageIfNotExists(rawDatasetId, writeConfig.streamName()); // When OVERWRITE mode, truncate the destination's raw table prior to syncing data if (writeConfig.syncMode() == DestinationSyncMode.OVERWRITE) { - bigQueryGcsOperations.truncateTableIfExists(datasetId, writeConfig.targetTableId(), writeConfig.tableSchema()); + // TODO: this might need special handling during the migration + bigQueryGcsOperations.truncateTableIfExists(rawDatasetId, writeConfig.targetTableId(), writeConfig.tableSchema()); } } LOGGER.info("Preparing airbyte_raw tables in destination completed."); }; } + private Map createFinalTables(boolean use1s1t, + final ParsedCatalog parsedCatalog, + final BigQueryDestinationHandler destinationHandler, + final BigQuerySqlGenerator sqlGenerator) + throws InterruptedException { + // TODO: share this code from BigQueryRecordConsumer + Map overwriteStreamsWithTmpTable = new HashMap<>(); + if (use1s1t) { + // For each stream, make sure that its corresponding final table exists. + for (StreamConfig stream : parsedCatalog.streams()) { + final Optional existingTable = destinationHandler.findExistingTable(stream.id()); + if (existingTable.isEmpty()) { + destinationHandler.execute(sqlGenerator.createTable(stream, "")); + if (stream.destinationSyncMode() == DestinationSyncMode.OVERWRITE) { + // We're creating this table for the first time. Write directly into it. + overwriteStreamsWithTmpTable.put(stream.id(), ""); + } + } else { + destinationHandler.execute(sqlGenerator.alterTable(stream, existingTable.get())); + if (stream.destinationSyncMode() == DestinationSyncMode.OVERWRITE) { + final BigInteger rowsInFinalTable = destinationHandler.getFinalTable(stream.id()).getNumRows(); + if (new BigInteger("0").equals(rowsInFinalTable)) { + // The table already exists but is empty. We'll load data incrementally. + // (this might be because the user ran a reset, which creates an empty table) + overwriteStreamsWithTmpTable.put(stream.id(), ""); + } else { + // We're working with an existing table. Write into a tmp table. We'll overwrite the table at the + // end of the sync. + overwriteStreamsWithTmpTable.put(stream.id(), OVERWRITE_TABLE_SUFFIX); + destinationHandler.execute(sqlGenerator.createTable(stream, OVERWRITE_TABLE_SUFFIX)); + } + } + } + } + } + return overwriteStreamsWithTmpTable; + } + /** * Flushes buffer data, writes to staging environment then proceeds to upload those same records to * destination table @@ -142,12 +245,14 @@ private OnStartFunction onStartFunction(final BigQueryStagingOperations bigQuery private FlushBufferFunction flushBufferFunction( final BigQueryStagingOperations bigQueryGcsOperations, final Map writeConfigs, - final ConfiguredAirbyteCatalog catalog) { + final ConfiguredAirbyteCatalog catalog, + final CheckedConsumer typerDeduper) { return (pair, writer) -> { LOGGER.info("Flushing buffer for stream {} ({}) to staging", pair.getName(), FileUtils.byteCountToDisplaySize(writer.getByteCount())); if (!writeConfigs.containsKey(pair)) { throw new IllegalArgumentException( - String.format("Message contained record from a stream that was not in the catalog. \ncatalog: %s", Jsons.serialize(catalog))); + String.format("Message contained record from a stream that was not in the catalog: %s.\nKeys: %s\ncatalog: %s", pair, + writeConfigs.keySet(), Jsons.serialize(catalog))); } final BigQueryWriteConfig writeConfig = writeConfigs.get(pair); @@ -164,6 +269,7 @@ private FlushBufferFunction flushBufferFunction( writeConfig.addStagedFile(stagedFile); bigQueryGcsOperations.copyIntoTableFromStage(datasetId, stream, writeConfig.targetTableId(), writeConfig.tableSchema(), List.of(stagedFile)); + typerDeduper.accept(new AirbyteStreamNameNamespacePair(writeConfig.streamName(), writeConfig.namespace())); } catch (final Exception e) { LOGGER.error("Failed to flush and commit buffer data into destination's raw table:", e); throw new RuntimeException("Failed to upload buffer to stage and commit to destination", e); @@ -176,10 +282,10 @@ private FlushBufferFunction flushBufferFunction( * * @param bigQueryGcsOperations collection of staging operations * @param writeConfigs configuration settings used to describe how to write data and where it exists - * @return */ private OnCloseFunction onCloseFunction(final BigQueryStagingOperations bigQueryGcsOperations, - final Map writeConfigs) { + final Map writeConfigs, + final CheckedConsumer replaceFinalTableConsumer) { return (hasFailed) -> { /* * Previously the hasFailed value was used to commit any remaining staged files into destination, @@ -190,9 +296,29 @@ private OnCloseFunction onCloseFunction(final BigQueryStagingOperations bigQuery LOGGER.info("Cleaning up destination started for {} streams", writeConfigs.size()); for (final BigQueryWriteConfig writeConfig : writeConfigs.values()) { bigQueryGcsOperations.dropStageIfExists(writeConfig.datasetId(), writeConfig.streamName()); + // replace final table + replaceFinalTableConsumer.accept(writeConfig); } LOGGER.info("Cleaning up destination completed."); }; } + private CheckedConsumer getReplaceFinalTableConsumer(boolean use1s1t, + final BigQuerySqlGenerator sqlGenerator, + final BigQueryDestinationHandler destinationHandler, + final Map overwriteStreamsWithTmpTable, + final ParsedCatalog parsedCatalog) { + return (writeConfig) -> { + final var streamConfig = parsedCatalog.getStream(writeConfig.namespace(), writeConfig.streamName()); + if (use1s1t && DestinationSyncMode.OVERWRITE.equals(writeConfig.syncMode())) { + LOGGER.info("Overwriting final table with tmp table"); + final Optional overwriteFinalTable = + sqlGenerator.overwriteFinalTable(overwriteStreamsWithTmpTable.get(streamConfig.id()), streamConfig); + if (overwriteFinalTable.isPresent()) { + destinationHandler.execute(overwriteFinalTable.get()); + } + } + }; + } + } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryUtils.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryUtils.java index 8eebe36f2763..57fe09be3282 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryUtils.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/BigQueryUtils.java @@ -38,8 +38,8 @@ import io.airbyte.commons.exceptions.ConfigErrorException; import io.airbyte.commons.json.Jsons; import io.airbyte.integrations.base.JavaBaseConstants; +import io.airbyte.integrations.base.TypingAndDedupingFlag; import io.airbyte.integrations.destination.gcs.GcsDestinationConfig; -import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream; import io.airbyte.protocol.models.v0.DestinationSyncMode; import java.time.Instant; import java.time.LocalDateTime; @@ -203,12 +203,14 @@ public static Table createTable(final BigQuery bigquery, final String datasetNam */ static void createPartitionedTableIfNotExists(final BigQuery bigquery, final TableId tableId, final Schema schema) { try { + final var chunkingColumn = + TypingAndDedupingFlag.isDestinationV2() ? JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT : JavaBaseConstants.COLUMN_NAME_EMITTED_AT; final TimePartitioning partitioning = TimePartitioning.newBuilder(TimePartitioning.Type.DAY) - .setField(JavaBaseConstants.COLUMN_NAME_EMITTED_AT) + .setField(chunkingColumn) .build(); final Clustering clustering = Clustering.newBuilder() - .setFields(ImmutableList.of(JavaBaseConstants.COLUMN_NAME_EMITTED_AT)) + .setFields(ImmutableList.of(chunkingColumn)) .build(); final StandardTableDefinition tableDefinition = @@ -221,6 +223,7 @@ static void createPartitionedTableIfNotExists(final BigQuery bigquery, final Tab final Table table = bigquery.getTable(tableInfo.getTableId()); if (table != null && table.exists()) { + // TODO: Handle migration from v1 -> v2 LOGGER.info("Partitioned table ALREADY EXISTS: {}", tableId); } else { bigquery.create(tableInfo); @@ -353,15 +356,6 @@ private static String getFormattedBigQueryDateTime(final String dateTimeValue) { : null); } - /** - * @return BigQuery dataset ID - */ - public static String getSchema(final JsonNode config, final ConfiguredAirbyteStream stream) { - final String srcNamespace = stream.getStream().getNamespace(); - final String schemaName = srcNamespace == null ? getDatasetId(config) : srcNamespace; - return sanitizeDatasetId(schemaName); - } - public static String sanitizeDatasetId(final String datasetId) { return NAME_TRANSFORMER.getNamespace(datasetId); } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/formatter/BigQueryRecordFormatter.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/formatter/BigQueryRecordFormatter.java index 1d8048330313..a6e7c0d48954 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/formatter/BigQueryRecordFormatter.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/formatter/BigQueryRecordFormatter.java @@ -46,6 +46,16 @@ protected JsonNode formatJsonSchema(final JsonNode jsonSchema) { return jsonSchema; }; + /*** + * To write to a JSON type, Standard inserts needs the Json to be an object. For Avro however, it + * needs to be a string. The column in the schema remains JSON regardless. + * + * @return whether to use an object for the formatting of the record. + */ + protected boolean useObjectForData() { + return true; + } + public abstract JsonNode formatRecord(AirbyteRecordMessage recordMessage); public Schema getBigQuerySchema() { diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/formatter/DefaultBigQueryRecordFormatter.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/formatter/DefaultBigQueryRecordFormatter.java index bd6756e94b08..2456d8124856 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/formatter/DefaultBigQueryRecordFormatter.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/formatter/DefaultBigQueryRecordFormatter.java @@ -11,8 +11,10 @@ import com.google.cloud.bigquery.StandardSQLTypeName; import io.airbyte.commons.json.Jsons; import io.airbyte.integrations.base.JavaBaseConstants; +import io.airbyte.integrations.base.TypingAndDedupingFlag; import io.airbyte.integrations.destination.StandardNameTransformer; import io.airbyte.protocol.models.v0.AirbyteRecordMessage; +import java.util.HashMap; import java.util.Map; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -23,21 +25,42 @@ */ public class DefaultBigQueryRecordFormatter extends BigQueryRecordFormatter { - private static final com.google.cloud.bigquery.Schema SCHEMA = com.google.cloud.bigquery.Schema.of( + public static final com.google.cloud.bigquery.Schema SCHEMA = com.google.cloud.bigquery.Schema.of( Field.of(JavaBaseConstants.COLUMN_NAME_AB_ID, StandardSQLTypeName.STRING), Field.of(JavaBaseConstants.COLUMN_NAME_EMITTED_AT, StandardSQLTypeName.TIMESTAMP), Field.of(JavaBaseConstants.COLUMN_NAME_DATA, StandardSQLTypeName.STRING)); + public static final com.google.cloud.bigquery.Schema SCHEMA_V2 = com.google.cloud.bigquery.Schema.of( + Field.of(JavaBaseConstants.COLUMN_NAME_AB_RAW_ID, StandardSQLTypeName.STRING), + Field.of(JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT, StandardSQLTypeName.TIMESTAMP), + Field.of(JavaBaseConstants.COLUMN_NAME_AB_LOADED_AT, StandardSQLTypeName.TIMESTAMP), + Field.of(JavaBaseConstants.COLUMN_NAME_DATA, StandardSQLTypeName.JSON)); + public DefaultBigQueryRecordFormatter(JsonNode jsonSchema, StandardNameTransformer namingResolver) { super(jsonSchema, namingResolver); } @Override public JsonNode formatRecord(AirbyteRecordMessage recordMessage) { - return Jsons.jsonNode(Map.of( - JavaBaseConstants.COLUMN_NAME_AB_ID, UUID.randomUUID().toString(), - JavaBaseConstants.COLUMN_NAME_EMITTED_AT, getEmittedAtField(recordMessage), - JavaBaseConstants.COLUMN_NAME_DATA, getData(recordMessage))); + if (TypingAndDedupingFlag.isDestinationV2()) { + // Map.of has a @NonNull requirement, so creating a new Hash map + final HashMap destinationV2record = new HashMap<>(); + destinationV2record.put(JavaBaseConstants.COLUMN_NAME_AB_RAW_ID, UUID.randomUUID().toString()); + destinationV2record.put(JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT, getEmittedAtField(recordMessage)); + destinationV2record.put(JavaBaseConstants.COLUMN_NAME_AB_LOADED_AT, null); + if (useObjectForData()) { + destinationV2record.put(JavaBaseConstants.COLUMN_NAME_DATA, + StandardNameTransformer.formatJsonPath(recordMessage.getData())); + } else { + destinationV2record.put(JavaBaseConstants.COLUMN_NAME_DATA, getData(recordMessage)); + } + return Jsons.jsonNode(destinationV2record); + } else { + return Jsons.jsonNode(Map.of( + JavaBaseConstants.COLUMN_NAME_AB_ID, UUID.randomUUID().toString(), + JavaBaseConstants.COLUMN_NAME_EMITTED_AT, getEmittedAtField(recordMessage), + JavaBaseConstants.COLUMN_NAME_DATA, getData(recordMessage))); + } } protected Object getEmittedAtField(AirbyteRecordMessage recordMessage) { @@ -54,7 +77,11 @@ protected Object getData(AirbyteRecordMessage recordMessage) { @Override public Schema getBigQuerySchema(JsonNode jsonSchema) { - return SCHEMA; + if (TypingAndDedupingFlag.isDestinationV2()) { + return SCHEMA_V2; + } else { + return SCHEMA; + } } } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/formatter/GcsAvroBigQueryRecordFormatter.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/formatter/GcsAvroBigQueryRecordFormatter.java index 44425f33bb28..08e284f26878 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/formatter/GcsAvroBigQueryRecordFormatter.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/formatter/GcsAvroBigQueryRecordFormatter.java @@ -17,6 +17,11 @@ public GcsAvroBigQueryRecordFormatter(JsonNode jsonSchema, StandardNameTransform super(jsonSchema, namingResolver); } + @Override + protected boolean useObjectForData() { + return false; + } + @Override protected Object getEmittedAtField(AirbyteRecordMessage recordMessage) { return recordMessage.getEmittedAt(); diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.java new file mode 100644 index 000000000000..287d1bd573d1 --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.java @@ -0,0 +1,49 @@ +/* + * 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.QueryJobConfiguration; +import com.google.cloud.bigquery.Table; +import com.google.cloud.bigquery.TableDefinition; +import com.google.cloud.bigquery.TableId; +import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator.StreamId; +import java.util.Optional; +import java.util.UUID; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +// TODO this stuff almost definitely exists somewhere else in our codebase. +public class BigQueryDestinationHandler { + + private static final Logger LOGGER = LoggerFactory.getLogger(BigQueryDestinationHandler.class); + + private final BigQuery bq; + + public BigQueryDestinationHandler(final BigQuery bq) { + this.bq = bq; + } + + public Optional findExistingTable(StreamId id) { + final Table table = bq.getTable(id.finalNamespace(), id.finalName()); + return Optional.ofNullable(table).map(Table::getDefinition); + } + + public Table getFinalTable(StreamId id) { + return bq.getTable(TableId.of(id.finalNamespace(), id.finalName())); + } + + public void execute(final String sql) throws InterruptedException { + if ("".equals(sql)) { + return; + } + final UUID queryId = UUID.randomUUID(); + LOGGER.info("Executing sql {}: {}", queryId, sql); + long start = System.currentTimeMillis(); + bq.query(QueryJobConfiguration.newBuilder(sql).build()); + LOGGER.info("Completed sql {} in {} ms", queryId, System.currentTimeMillis() - start); + } + +} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.java new file mode 100644 index 000000000000..c7885eb3c278 --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.java @@ -0,0 +1,481 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.destination.bigquery.typing_deduping; + +import static java.util.stream.Collectors.joining; + +import com.google.cloud.bigquery.StandardSQLTypeName; +import com.google.cloud.bigquery.StandardTableDefinition; +import com.google.cloud.bigquery.TableDefinition; +import com.google.common.annotations.VisibleForTesting; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.AirbyteProtocolType; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.Array; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.OneOf; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.Struct; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.UnsupportedOneOf; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteTypeUtils; +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser.StreamConfig; +import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator; +import io.airbyte.integrations.destination.bigquery.BigQuerySQLNameTransformer; +import io.airbyte.protocol.models.v0.DestinationSyncMode; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.commons.text.StringSubstitutor; + +public class BigQuerySqlGenerator implements SqlGenerator { + + public static final String QUOTE = "`"; + private static final BigQuerySQLNameTransformer nameTransformer = new BigQuerySQLNameTransformer(); + + private final ColumnId CDC_DELETED_AT_COLUMN = buildColumnId("_ab_cdc_deleted_at"); + + @Override + public StreamId buildStreamId(final String namespace, final String name, final String rawNamespaceOverride) { + return new StreamId( + // TODO is this correct? + nameTransformer.getNamespace(namespace), + nameTransformer.convertStreamName(name), + nameTransformer.getNamespace(rawNamespaceOverride), + nameTransformer.convertStreamName(namespace + "_" + name), + namespace, + name); + } + + @Override + public ColumnId buildColumnId(final String name) { + String quotedName = name; + + // Column names aren't allowed to start with certain strings. Prepend an underscore if this happens. + final List invalidColumnPrefixes = List.of( + "_table_", + "_file_", + "_partition_", + "_row_timestamp_", + "__root__", + "_colidentifier_" + ); + String canonicalized = name.toLowerCase(); + // Bigquery columns are case insensitive, so do all our validation on the lowercased name + if (invalidColumnPrefixes.stream().anyMatch(prefix -> name.toLowerCase().startsWith(prefix))) { + quotedName = "_" + quotedName; + canonicalized = "_" + canonicalized; + } + + // TODO this is probably wrong + return new ColumnId(nameTransformer.getIdentifier(quotedName), name, canonicalized); + } + + public StandardSQLTypeName toDialectType(final AirbyteType type) { + // switch pattern-matching is still in preview at language level 17 :( + if (type instanceof final AirbyteProtocolType p) { + return toDialectType(p); + } else if (type instanceof Struct) { + return StandardSQLTypeName.JSON; + } else if (type instanceof Array) { + return StandardSQLTypeName.JSON; + } else if (type instanceof UnsupportedOneOf) { + return StandardSQLTypeName.JSON; + } else if (type instanceof final OneOf o) { + final AirbyteType typeWithPrecedence = AirbyteTypeUtils.chooseOneOfType(o); + final StandardSQLTypeName dialectType; + if ((typeWithPrecedence instanceof Struct) || (typeWithPrecedence instanceof Array)) { + dialectType = StandardSQLTypeName.JSON; + } else { + dialectType = toDialectType((AirbyteProtocolType) typeWithPrecedence); + } + return dialectType; + } + + // Literally impossible; AirbyteType is a sealed interface. + throw new IllegalArgumentException("Unsupported AirbyteType: " + type); + } + + private String extractAndCast(final ColumnId column, final AirbyteType airbyteType) { + if (airbyteType instanceof OneOf o) { + // This is guaranteed to not be a OneOf, so we won't recurse infinitely + final AirbyteType chosenType = AirbyteTypeUtils.chooseOneOfType(o); + return extractAndCast(column, chosenType); + } else if (airbyteType instanceof Struct) { + // We need to validate that the struct is actually a struct. + // Note that struct columns are actually nullable in two ways. For a column `foo`: + // {foo: null} and {} are both valid, and are both written to the final table as a SQL NULL (_not_ a + // JSON null). + // JSON_QUERY(JSON'{}', '$.foo') returns a SQL null. + // JSON_QUERY(JSON'{"foo": null}', '$.foo') returns a JSON null. + return new StringSubstitutor(Map.of("column_name", column.originalName())).replace( + """ + CASE + WHEN JSON_QUERY(`_airbyte_data`, '$.${column_name}') IS NULL + OR JSON_TYPE(JSON_QUERY(`_airbyte_data`, '$.${column_name}')) != 'object' + THEN NULL + ELSE JSON_QUERY(`_airbyte_data`, '$.${column_name}') + END + """); + } else if (airbyteType instanceof Array) { + // Much like the Struct case above, arrays need special handling. + return new StringSubstitutor(Map.of("column_name", column.originalName())).replace( + """ + CASE + WHEN JSON_QUERY(`_airbyte_data`, '$.${column_name}') IS NULL + OR JSON_TYPE(JSON_QUERY(`_airbyte_data`, '$.${column_name}')) != 'array' + THEN NULL + ELSE JSON_QUERY(`_airbyte_data`, '$.${column_name}') + END + """); + } else if (airbyteType instanceof UnsupportedOneOf || airbyteType == AirbyteProtocolType.UNKNOWN) { + // JSON_VALUE converts JSON types to native SQL types (int64, string, etc.) + // We use JSON_QUERY rather than JSON_VALUE so that we can extract a JSON-typed value. + // This is to avoid needing to convert the raw SQL type back into JSON. + return "JSON_QUERY(`_airbyte_data`, '$." + column.originalName() + "')"; + } else { + final StandardSQLTypeName dialectType = toDialectType(airbyteType); + return "SAFE_CAST(JSON_VALUE(`_airbyte_data`, '$." + column.originalName() + "') as " + dialectType.name() + ")"; + } + } + + public StandardSQLTypeName toDialectType(final AirbyteProtocolType airbyteProtocolType) { + return switch (airbyteProtocolType) { + // TODO doublecheck these + case STRING -> StandardSQLTypeName.STRING; + case NUMBER -> StandardSQLTypeName.NUMERIC; + case INTEGER -> StandardSQLTypeName.INT64; + case BOOLEAN -> StandardSQLTypeName.BOOL; + case TIMESTAMP_WITH_TIMEZONE -> StandardSQLTypeName.TIMESTAMP; + case TIMESTAMP_WITHOUT_TIMEZONE -> StandardSQLTypeName.DATETIME; + case TIME_WITH_TIMEZONE -> StandardSQLTypeName.STRING; + case TIME_WITHOUT_TIMEZONE -> StandardSQLTypeName.TIME; + case DATE -> StandardSQLTypeName.DATE; + case UNKNOWN -> StandardSQLTypeName.JSON; + }; + } + + @Override + public String createTable(final StreamConfig stream, final String suffix) { + final String columnDeclarations = stream.columns().entrySet().stream() + .map(column -> column.getKey().name(QUOTE) + " " + toDialectType(column.getValue()).name()) + .collect(joining(",\n")); + final String clusterConfig; + if (stream.destinationSyncMode() == DestinationSyncMode.APPEND_DEDUP) { + // We're doing deduping, therefore we have a primary key. + // Cluster on all the PK columns, and also extracted_at. + clusterConfig = stream.primaryKey().stream().map(columnId -> columnId.name(QUOTE)).collect(joining(",")) + ", _airbyte_extracted_at"; + } else { + // Otherwise just cluser on extracted_at. + clusterConfig = "_airbyte_extracted_at"; + } + + return new StringSubstitutor(Map.of( + "final_namespace", stream.id().finalNamespace(QUOTE), + "final_table_id", stream.id().finalTableId(suffix, QUOTE), + "column_declarations", columnDeclarations, + "cluster_config", clusterConfig)).replace( + """ + CREATE SCHEMA IF NOT EXISTS ${final_namespace}; + + CREATE TABLE ${final_table_id} ( + _airbyte_raw_id STRING NOT NULL, + _airbyte_extracted_at TIMESTAMP NOT NULL, + _airbyte_meta JSON NOT NULL, + ${column_declarations} + ) + PARTITION BY (DATE_TRUNC(_airbyte_extracted_at, DAY)) + CLUSTER BY ${cluster_config} + """); + } + + @Override + public String alterTable(final StreamConfig stream, + final TableDefinition existingTable) { + if (existingTable instanceof final StandardTableDefinition s) { + // TODO check if clustering/partitioning config is different from what we want, do something to + // handle it + // iirc this will depend on the stream (destination?) sync mode + cursor + pk name + if (s.getClustering() != null) { + + } + } else { + // TODO idk + } + /* + * TODO maybe we do something like this? CREATE OR REPLACE TABLE ${final_table_id} AS ( SELECT + * _airbyte_raw_id, _airbyte_extracted_at, _airbyte_meta, -- cast columns when needed CAST(col1 AS + * new_type) AS col1, -- some columns will not change at all col2, -- new columns default to null + * NULL as col3 ... FROM ${final_table_id} ) + * + * This has a few advantages: * bypasses the whole "you can only alter certain column types" problem + * * preserves column ordering + * + * But it does mean that we have to rewrite the entire table, which really sucks. But maybe that's + * fine, since it only happens on schema change? And it's presumably no worse than a soft reset. + */ + return ""; + } + + @Override + public String updateTable(final String finalSuffix, final StreamConfig stream) { + String validatePrimaryKeys = ""; + if (stream.destinationSyncMode() == DestinationSyncMode.APPEND_DEDUP) { + validatePrimaryKeys = validatePrimaryKeys(stream.id(), stream.primaryKey(), stream.columns()); + } + final String insertNewRecords = insertNewRecords(stream.id(), finalSuffix, stream.columns(), stream.destinationSyncMode()); + String dedupFinalTable = ""; + String cdcDeletes = ""; + String dedupRawTable = ""; + if (stream.destinationSyncMode() == DestinationSyncMode.APPEND_DEDUP) { + dedupRawTable = dedupRawTable(stream.id(), finalSuffix); + // If we're in dedup mode, then we must have a cursor + dedupFinalTable = dedupFinalTable(stream.id(), finalSuffix, stream.primaryKey(), stream.cursor().get()); + cdcDeletes = cdcDeletes(stream, finalSuffix, stream.columns()); + } + final String commitRawTable = commitRawTable(stream.id()); + + return new StringSubstitutor(Map.of( + "validate_primary_keys", validatePrimaryKeys, + "insert_new_records", insertNewRecords, + "dedup_final_table", dedupFinalTable, + "cdc_deletes", cdcDeletes, + "dedupe_raw_table", dedupRawTable, + "commit_raw_table", commitRawTable)).replace( + """ + DECLARE missing_pk_count INT64; + + BEGIN TRANSACTION; + + ${validate_primary_keys} + + ${insert_new_records} + + ${dedup_final_table} + + ${dedupe_raw_table} + + ${cdc_deletes} + + ${commit_raw_table} + + COMMIT TRANSACTION; + """); + } + + @VisibleForTesting + String validatePrimaryKeys(final StreamId id, + final List primaryKeys, + final LinkedHashMap streamColumns) { + final String pkNullChecks = primaryKeys.stream().map( + pk -> { + final String jsonExtract = extractAndCast(pk, streamColumns.get(pk)); + return "AND " + jsonExtract + " IS NULL"; + }).collect(joining("\n")); + + return new StringSubstitutor(Map.of( + "raw_table_id", id.rawTableId(QUOTE), + "pk_null_checks", pkNullChecks)).replace( + """ + SET missing_pk_count = ( + SELECT COUNT(1) + FROM ${raw_table_id} + WHERE + `_airbyte_loaded_at` IS NULL + ${pk_null_checks} + ); + + IF missing_pk_count > 0 THEN + RAISE USING message = FORMAT("Raw table has %s rows missing a primary key", CAST(missing_pk_count AS STRING)); + END IF + ;"""); + } + + @VisibleForTesting + String insertNewRecords(final StreamId id, final String finalSuffix, final LinkedHashMap streamColumns, DestinationSyncMode destinationSyncMode) { + final String columnCasts = streamColumns.entrySet().stream().map( + col -> extractAndCast(col.getKey(), col.getValue()) + " as " + col.getKey().name(QUOTE) + ",") + .collect(joining("\n")); + final String columnErrors = streamColumns.entrySet().stream().map( + col -> new StringSubstitutor(Map.of( + "raw_col_name", col.getKey().originalName(), + "col_type", toDialectType(col.getValue()).name(), + "json_extract", extractAndCast(col.getKey(), col.getValue()))).replace( + """ + CASE + WHEN (JSON_QUERY(`_airbyte_data`, '$.${raw_col_name}') IS NOT NULL) + AND (JSON_TYPE(JSON_QUERY(`_airbyte_data`, '$.${raw_col_name}')) != 'null') + AND (${json_extract} IS NULL) + THEN ["Problem with `${raw_col_name}`"] + ELSE [] + END""")) + .collect(joining(",\n")); + final String columnList = streamColumns.keySet().stream().map(quotedColumnId -> quotedColumnId.name(QUOTE) + ",").collect(joining("\n")); + final String deletionClause; + if (destinationSyncMode == DestinationSyncMode.APPEND_DEDUP && streamColumns.keySet().stream().anyMatch(col -> "_ab_cdc_deleted_at".equals(col.originalName()))) { + deletionClause = """ + AND ( + JSON_QUERY(`_airbyte_data`, '$._ab_cdc_deleted_at') IS NULL + OR JSON_TYPE(JSON_QUERY(`_airbyte_data`, '$._ab_cdc_deleted_at')) = 'null' + ) + """; + } else { + deletionClause = ""; + } + + String cdcConditionalOrIncludeStatement = ""; + if (streamColumns.containsKey(CDC_DELETED_AT_COLUMN)){ + cdcConditionalOrIncludeStatement = """ + OR ( + _airbyte_loaded_at IS NOT NULL + AND JSON_VALUE(`_airbyte_data`, '$._ab_cdc_deleted_at') IS NOT NULL + ) + """; + } + + return new StringSubstitutor(Map.of( + "raw_table_id", id.rawTableId(QUOTE), + "final_table_id", id.finalTableId(finalSuffix, QUOTE), + "column_casts", columnCasts, + "column_errors", columnErrors, + "cdcConditionalOrIncludeStatement", cdcConditionalOrIncludeStatement, + "column_list", columnList)).replace( + """ + INSERT INTO ${final_table_id} + ( + ${column_list} + _airbyte_meta, + _airbyte_raw_id, + _airbyte_extracted_at + ) + WITH intermediate_data AS ( + SELECT + ${column_casts} + array_concat( + ${column_errors} + ) as _airbyte_cast_errors, + _airbyte_raw_id, + _airbyte_extracted_at + FROM ${raw_table_id} + WHERE + _airbyte_loaded_at IS NULL + ${cdcConditionalOrIncludeStatement} + ) + SELECT + ${column_list} + to_json(struct(_airbyte_cast_errors AS errors)) AS _airbyte_meta, + _airbyte_raw_id, + _airbyte_extracted_at + FROM intermediate_data;"""); + } + + @VisibleForTesting + String dedupFinalTable(final StreamId id, + final String finalSuffix, + final List primaryKey, + final ColumnId cursor) { + final String pkList = primaryKey.stream().map(columnId -> columnId.name(QUOTE)).collect(joining(",")); + + return new StringSubstitutor(Map.of( + "final_table_id", id.finalTableId(finalSuffix, QUOTE), + "pk_list", pkList, + "cursor_name", cursor.name(QUOTE)) + ).replace( + """ + DELETE FROM ${final_table_id} + WHERE + `_airbyte_raw_id` IN ( + SELECT `_airbyte_raw_id` FROM ( + SELECT `_airbyte_raw_id`, row_number() OVER ( + PARTITION BY ${pk_list} ORDER BY ${cursor_name} DESC, `_airbyte_extracted_at` DESC + ) as row_number FROM ${final_table_id} + ) + WHERE row_number != 1 + ) + ;"""); + } + + @VisibleForTesting + String cdcDeletes(final StreamConfig stream, + final String finalSuffix, + final LinkedHashMap streamColumns) { + + if (stream.destinationSyncMode() != DestinationSyncMode.APPEND_DEDUP){ + return ""; + } + + if (!streamColumns.containsKey(CDC_DELETED_AT_COLUMN)){ + return ""; + } + + final String pkList = stream.primaryKey().stream().map(columnId -> columnId.name(QUOTE)).collect(joining(",")); + String pkCasts = stream.primaryKey().stream().map(pk -> extractAndCast(pk, streamColumns.get(pk))).collect(joining(",\n")); + + // we want to grab IDs for deletion from the raw table (not the final table itself) to hand out-of-order record insertions after the delete has been registered + return new StringSubstitutor(Map.of( + "final_table_id", stream.id().finalTableId(finalSuffix, QUOTE), + "raw_table_id", stream.id().rawTableId(QUOTE), + "pk_list", pkList, + "pk_extracts", pkCasts, + "quoted_cdc_delete_column", QUOTE + "_ab_cdc_deleted_at" + QUOTE) + ).replace( + """ + DELETE FROM ${final_table_id} + WHERE + (${pk_list}) IN ( + SELECT ( + ${pk_extracts} + ) + FROM ${raw_table_id} + WHERE + JSON_VALUE(`_airbyte_data`, '$._ab_cdc_deleted_at') IS NOT NULL + ) + ;""" + ); + } + + @VisibleForTesting + String dedupRawTable(final StreamId id, final String finalSuffix) { + return new StringSubstitutor(Map.of( + "raw_table_id", id.rawTableId(QUOTE), + "final_table_id", id.finalTableId(finalSuffix, QUOTE))).replace( + // Note that this leaves _all_ deletion records in the raw table. We _could_ clear them out, but it + // would be painful, + // and it only matters in a few edge cases. + """ + DELETE FROM + ${raw_table_id} + WHERE + `_airbyte_raw_id` NOT IN ( + SELECT `_airbyte_raw_id` FROM ${final_table_id} + ) + ;"""); + } + + @VisibleForTesting + String commitRawTable(final StreamId id) { + return new StringSubstitutor(Map.of( + "raw_table_id", id.rawTableId(QUOTE))).replace( + """ + UPDATE ${raw_table_id} + SET `_airbyte_loaded_at` = CURRENT_TIMESTAMP() + WHERE `_airbyte_loaded_at` IS NULL + ;"""); + } + + @Override + public Optional overwriteFinalTable(final String finalSuffix, final StreamConfig stream) { + if (stream.destinationSyncMode() == DestinationSyncMode.OVERWRITE && finalSuffix.length() > 0) { + return Optional.of(new StringSubstitutor(Map.of( + "final_table_id", stream.id().finalTableId(QUOTE), + "tmp_final_table", stream.id().finalTableId(finalSuffix, QUOTE), + "real_final_table", stream.id().finalName(QUOTE))).replace( + """ + DROP TABLE IF EXISTS ${final_table_id}; + ALTER TABLE ${tmp_final_table} RENAME TO ${real_final_table}; + """)); + } else { + return Optional.empty(); + } + } + +} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/AbstractBigQueryUploader.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/AbstractBigQueryUploader.java index f61a2a9bd023..390e2f7fdfd8 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/AbstractBigQueryUploader.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/AbstractBigQueryUploader.java @@ -14,9 +14,13 @@ import com.google.cloud.bigquery.JobInfo.WriteDisposition; import com.google.cloud.bigquery.QueryJobConfiguration; import com.google.cloud.bigquery.Schema; +import com.google.cloud.bigquery.StandardTableDefinition; +import com.google.cloud.bigquery.Table; import com.google.cloud.bigquery.TableId; +import com.google.cloud.bigquery.TableInfo; import io.airbyte.commons.string.Strings; import io.airbyte.integrations.base.JavaBaseConstants; +import io.airbyte.integrations.base.TypingAndDedupingFlag; import io.airbyte.integrations.destination.bigquery.BigQueryUtils; import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter; import io.airbyte.integrations.destination.s3.writer.DestinationWriter; @@ -38,6 +42,7 @@ public abstract class AbstractBigQueryUploader { protected final T writer; protected final BigQuery bigQuery; protected final BigQueryRecordFormatter recordFormatter; + protected final boolean use1s1t; AbstractBigQueryUploader(final TableId table, final TableId tmpTable, @@ -45,6 +50,7 @@ public abstract class AbstractBigQueryUploader { final WriteDisposition syncMode, final BigQuery bigQuery, final BigQueryRecordFormatter recordFormatter) { + this.use1s1t = TypingAndDedupingFlag.isDestinationV2(); this.table = table; this.tmpTable = tmpTable; this.writer = writer; @@ -96,9 +102,13 @@ public void close(final boolean hasFailed, final Consumer output protected void uploadData(final Consumer outputRecordCollector, final AirbyteMessage lastStateMessage) throws Exception { try { - LOGGER.info("Uploading data from the tmp table {} to the source table {}.", tmpTable.getTable(), table.getTable()); - uploadDataToTableFromTmpTable(); - LOGGER.info("Data is successfully loaded to the source table {}!", table.getTable()); + if (!use1s1t) { + // This only needs to happen if we actually wrote to a tmp table. + LOGGER.info("Uploading data from the tmp table {} to the source table {}.", tmpTable.getTable(), table.getTable()); + uploadDataToTableFromTmpTable(); + LOGGER.info("Data is successfully loaded to the source table {}!", table.getTable()); + } + outputRecordCollector.accept(lastStateMessage); LOGGER.info("Final state message is accepted."); } catch (final Exception e) { @@ -109,6 +119,18 @@ protected void uploadData(final Consumer outputRecordCollector, } } + public void createRawTable() { + // Ensure that this table exists. + // TODO alter an existing raw table? + final Table rawTable = bigQuery.getTable(table); + if (rawTable == null) { + LOGGER.info("Creating raw table {}.", table); + bigQuery.create(TableInfo.newBuilder(table, StandardTableDefinition.of(recordFormatter.getBigQuerySchema())).build()); + } else { + LOGGER.info("Found raw table {}.", rawTable.getTableId()); + } + } + protected void dropTmpTable() { try { // clean up tmp tables; diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/BigQueryDirectUploader.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/BigQueryDirectUploader.java index 3bd481c03b58..81a4641395ff 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/BigQueryDirectUploader.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/BigQueryDirectUploader.java @@ -4,7 +4,9 @@ package io.airbyte.integrations.destination.bigquery.uploader; -import com.google.cloud.bigquery.*; +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.JobInfo; +import com.google.cloud.bigquery.TableId; import io.airbyte.integrations.destination.bigquery.BigQueryUtils; import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter; import io.airbyte.integrations.destination.bigquery.writer.BigQueryTableWriter; diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/BigQueryUploaderFactory.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/BigQueryUploaderFactory.java index 67d52a68334c..3ee2fdafa23b 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/BigQueryUploaderFactory.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/BigQueryUploaderFactory.java @@ -20,6 +20,7 @@ import com.google.cloud.bigquery.TableId; import com.google.cloud.bigquery.WriteChannelConfiguration; import io.airbyte.commons.exceptions.ConfigErrorException; +import io.airbyte.integrations.base.TypingAndDedupingFlag; import io.airbyte.integrations.destination.bigquery.BigQueryUtils; import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter; import io.airbyte.integrations.destination.bigquery.uploader.config.UploaderConfig; @@ -31,9 +32,13 @@ import java.sql.Timestamp; import java.util.HashSet; import java.util.Set; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class BigQueryUploaderFactory { + private static final Logger LOGGER = LoggerFactory.getLogger(BigQueryUploaderFactory.class); + private static final String CONFIG_ERROR_MSG = """ Failed to write to destination schema. @@ -50,20 +55,27 @@ public class BigQueryUploaderFactory { public static AbstractBigQueryUploader getUploader(final UploaderConfig uploaderConfig) throws IOException { - final String schemaName = BigQueryUtils.getSchema(uploaderConfig.getConfig(), uploaderConfig.getConfigStream()); + final String dataset; + if (TypingAndDedupingFlag.isDestinationV2()) { + dataset = uploaderConfig.getParsedStream().id().rawNamespace(); + } else { + // This previously needed to handle null namespaces. That's now happening at the top of the + // connector, so we can assume namespace is non-null here. + dataset = BigQueryUtils.sanitizeDatasetId(uploaderConfig.getConfigStream().getStream().getNamespace()); + } final String datasetLocation = BigQueryUtils.getDatasetLocation(uploaderConfig.getConfig()); - final Set existingSchemas = new HashSet<>(); + final Set existingDatasets = new HashSet<>(); final BigQueryRecordFormatter recordFormatter = uploaderConfig.getFormatter(); final Schema bigQuerySchema = recordFormatter.getBigQuerySchema(); - final TableId targetTable = TableId.of(schemaName, uploaderConfig.getTargetTableName()); - final TableId tmpTable = TableId.of(schemaName, uploaderConfig.getTmpTableName()); + final TableId targetTable = TableId.of(dataset, uploaderConfig.getTargetTableName()); + final TableId tmpTable = TableId.of(dataset, uploaderConfig.getTmpTableName()); BigQueryUtils.createSchemaAndTableIfNeeded( uploaderConfig.getBigQuery(), - existingSchemas, - schemaName, + existingDatasets, + dataset, tmpTable, datasetLocation, bigQuerySchema); @@ -146,8 +158,10 @@ private static BigQueryDirectUploader getBigQueryDirectUploader( final String datasetLocation, final BigQueryRecordFormatter formatter) { // https://cloud.google.com/bigquery/docs/loading-data-local#loading_data_from_a_local_data_source + final TableId tableToWriteRawData = TypingAndDedupingFlag.isDestinationV2() ? targetTable : tmpTable; + LOGGER.info("Will write raw data to {} with schema {}", tableToWriteRawData, formatter.getBigQuerySchema()); final WriteChannelConfiguration writeChannelConfiguration = - WriteChannelConfiguration.newBuilder(tmpTable) + WriteChannelConfiguration.newBuilder(tableToWriteRawData) .setCreateDisposition(JobInfo.CreateDisposition.CREATE_IF_NEEDED) .setSchema(formatter.getBigQuerySchema()) .setFormatOptions(FormatOptions.json()) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/config/UploaderConfig.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/config/UploaderConfig.java index ab13465c9cc6..f6246989d1b4 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/config/UploaderConfig.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/config/UploaderConfig.java @@ -6,6 +6,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.google.cloud.bigquery.BigQuery; +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser.StreamConfig; import io.airbyte.integrations.destination.bigquery.BigQueryUtils; import io.airbyte.integrations.destination.bigquery.UploadingMethod; import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter; @@ -20,7 +21,15 @@ public class UploaderConfig { private JsonNode config; + /** + * Taken directly from the {@link ConfiguredAirbyteStream}, except if the namespace was null, we set + * it to the destination default namespace. + */ private ConfiguredAirbyteStream configStream; + /** + * Parsed directly from {@link #configStream}. + */ + private StreamConfig parsedStream; private String targetTableName; private String tmpTableName; private BigQuery bigQuery; diff --git a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.java b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.java index de865f5ad80e..9a318aca6a78 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.java @@ -30,6 +30,7 @@ import io.airbyte.commons.string.Strings; import io.airbyte.integrations.base.AirbyteMessageConsumer; import io.airbyte.integrations.base.Destination; +import io.airbyte.integrations.base.DestinationConfig; import io.airbyte.integrations.base.JavaBaseConstants; import io.airbyte.integrations.destination.NamingConventionTransformer; import io.airbyte.integrations.destination.gcs.GcsDestinationConfig; @@ -176,6 +177,8 @@ public static void beforeAll() throws IOException { // this config is also used for housekeeping (checking records, and cleaning up) config = BigQueryDestinationTestUtils.createConfig(CREDENTIALS_STANDARD_INSERT_PATH, datasetId); + DestinationConfig.initialize(config); + // all successful configs use the same project ID projectId = config.get(BigQueryConsts.CONFIG_PROJECT_ID).asText(); diff --git a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTestUtils.java b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTestUtils.java index cdc5e042078f..cac72e263a43 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTestUtils.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTestUtils.java @@ -21,8 +21,10 @@ import java.util.LinkedList; import java.util.List; import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class BigQueryDestinationTestUtils { + private static final Logger LOGGER = LoggerFactory.getLogger(BigQueryDestinationTestUtils.class); /** * Parse the config file and replace dataset with datasetId randomly generated by the test @@ -33,6 +35,7 @@ public class BigQueryDestinationTestUtils { * @throws IOException */ public static JsonNode createConfig(Path configFile, String datasetId) throws IOException { + LOGGER.info("Setting default dataset to {}", datasetId); final String tmpConfigAsString = Files.readString(configFile); final JsonNode tmpConfigJson = Jsons.deserialize(tmpConfigAsString); return Jsons.jsonNode(((ObjectNode) tmpConfigJson).put(BigQueryConsts.CONFIG_DATASET_ID, datasetId)); diff --git a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/BigQueryGcsDestinationAcceptanceTest.java b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/BigQueryGcsDestinationAcceptanceTest.java index 5f69f0ccee36..ca29799fd3cb 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/BigQueryGcsDestinationAcceptanceTest.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/BigQueryGcsDestinationAcceptanceTest.java @@ -11,6 +11,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.ObjectNode; import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.base.DestinationConfig; import io.airbyte.integrations.destination.gcs.GcsDestinationConfig; import io.airbyte.integrations.destination.record_buffer.FileBuffer; import io.airbyte.integrations.standardtest.destination.DestinationAcceptanceTest; @@ -40,6 +41,8 @@ protected void setup(TestDestinationEnv testEnv) throws Exception { secretsFile = Path.of("secrets/credentials-gcs-staging.json"); setUpBigQuery(); + DestinationConfig.initialize(config); + // the setup steps below are specific to GCS staging use case final GcsDestinationConfig gcsDestinationConfig = GcsDestinationConfig .getGcsDestinationConfig(BigQueryUtils.getGcsJsonNodeConfig(config)); diff --git a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/typing_deduping/AbstractBigQueryTypingDedupingTest.java b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/typing_deduping/AbstractBigQueryTypingDedupingTest.java new file mode 100644 index 000000000000..15fc029302b3 --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/typing_deduping/AbstractBigQueryTypingDedupingTest.java @@ -0,0 +1,68 @@ +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.TableId; +import com.google.cloud.bigquery.TableResult; +import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.base.destination.typing_deduping.BaseTypingDedupingTest; +import io.airbyte.integrations.destination.bigquery.BigQueryDestination; +import io.airbyte.integrations.destination.bigquery.BigQueryDestinationTestUtils; +import io.airbyte.integrations.destination.bigquery.BigQueryUtils; +import java.io.IOException; +import java.nio.file.Path; +import java.time.Instant; +import java.util.LinkedHashMap; +import java.util.List; + +public abstract class AbstractBigQueryTypingDedupingTest extends BaseTypingDedupingTest { + + private BigQuery bq; + + protected abstract String getConfigPath(); + + @Override + public JsonNode generateConfig() throws IOException { + final String datasetId = "typing_deduping_default_dataset" + getUniqueSuffix(); + JsonNode config = BigQueryDestinationTestUtils.createConfig(Path.of(getConfigPath()), datasetId); + bq = BigQueryDestination.getBigQuery(config); + return config; + } + + @Override + protected String getImageName() { + return "airbyte/destination-bigquery:dev"; + } + + @Override + protected List dumpRawTableRecords(String streamNamespace, String streamName) throws InterruptedException { + if (streamNamespace == null) { + streamNamespace = BigQueryUtils.getDatasetId(getConfig()); + } + TableResult result = bq.query(QueryJobConfiguration.of("SELECT * FROM airbyte." + streamNamespace + "_" + streamName)); + return BigQuerySqlGeneratorIntegrationTest.toJsonRecords(result); + } + + @Override + protected List dumpFinalTableRecords(String streamNamespace, String streamName) throws InterruptedException { + if (streamNamespace == null) { + streamNamespace = BigQueryUtils.getDatasetId(getConfig()); + } + TableResult result = bq.query(QueryJobConfiguration.of("SELECT * FROM " + streamNamespace + "." + streamName)); + return BigQuerySqlGeneratorIntegrationTest.toJsonRecords(result); + } + + @Override + protected void teardownStreamAndNamespace(String streamNamespace, String streamName) { + if (streamNamespace == null) { + streamNamespace = BigQueryUtils.getDatasetId(getConfig()); + } + // 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("airbyte", streamNamespace + "_" + streamName)); + bq.delete(DatasetId.of(streamNamespace), BigQuery.DatasetDeleteOption.deleteContents()); + } +} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryGcsTypingDedupingTest.java b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryGcsTypingDedupingTest.java new file mode 100644 index 000000000000..df201d6c687b --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryGcsTypingDedupingTest.java @@ -0,0 +1,9 @@ +package io.airbyte.integrations.destination.bigquery.typing_deduping; + +public class BigQueryGcsTypingDedupingTest extends AbstractBigQueryTypingDedupingTest { + + @Override + public String getConfigPath() { + return "secrets/credentials-1s1t-gcs.json"; + } +} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGeneratorIntegrationTest.java b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGeneratorIntegrationTest.java new file mode 100644 index 000000000000..eb0675d05bc1 --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGeneratorIntegrationTest.java @@ -0,0 +1,921 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.destination.bigquery.typing_deduping; + +import static com.google.cloud.bigquery.LegacySQLTypeName.legacySQLTypeName; +import static org.junit.jupiter.api.Assertions.*; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.BigQueryException; +import com.google.cloud.bigquery.DatasetInfo; +import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.Field.Mode; +import com.google.cloud.bigquery.FieldValue; +import com.google.cloud.bigquery.FieldValueList; +import com.google.cloud.bigquery.QueryJobConfiguration; +import com.google.cloud.bigquery.Schema; +import com.google.cloud.bigquery.StandardSQLTypeName; +import com.google.cloud.bigquery.Table; +import com.google.cloud.bigquery.TableResult; +import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.AirbyteProtocolType; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.Array; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.Struct; +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser.StreamConfig; +import io.airbyte.integrations.base.destination.typing_deduping.RecordDiffer; +import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator.ColumnId; +import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator.StreamId; +import io.airbyte.integrations.destination.bigquery.BigQueryDestination; +import io.airbyte.protocol.models.v0.DestinationSyncMode; +import io.airbyte.protocol.models.v0.SyncMode; +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.text.StringSubstitutor; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; +import org.junit.jupiter.api.parallel.ExecutionMode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +// TODO write test case for multi-column PK +@Execution(ExecutionMode.CONCURRENT) +public class BigQuerySqlGeneratorIntegrationTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(BigQuerySqlGeneratorIntegrationTest.class); + private static final BigQuerySqlGenerator GENERATOR = new BigQuerySqlGenerator(); + public static final ColumnId ID_COLUMN = GENERATOR.buildColumnId("id"); + public static final List PRIMARY_KEY = List.of(ID_COLUMN); + public static final ColumnId CURSOR = GENERATOR.buildColumnId("updated_at"); + public static final ColumnId CDC_CURSOR = GENERATOR.buildColumnId("_ab_cdc_lsn"); + public static final RecordDiffer DIFFER = new RecordDiffer( + Pair.of("id", AirbyteProtocolType.INTEGER), + Pair.of("updated_at", AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE), + Pair.of("_ab_cdc_lsn", AirbyteProtocolType.INTEGER) + ); + public static final String QUOTE = "`"; + private static final LinkedHashMap COLUMNS; + private static final LinkedHashMap CDC_COLUMNS; + + private static BigQuery bq; + + private String testDataset; + private StreamId streamId; + + static { + COLUMNS = new LinkedHashMap<>(); + COLUMNS.put(ID_COLUMN, AirbyteProtocolType.INTEGER); + COLUMNS.put(CURSOR, AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE); + COLUMNS.put(GENERATOR.buildColumnId("struct"), new Struct(new LinkedHashMap<>())); + COLUMNS.put(GENERATOR.buildColumnId("array"), new Array(AirbyteProtocolType.UNKNOWN)); + COLUMNS.put(GENERATOR.buildColumnId("string"), AirbyteProtocolType.STRING); + COLUMNS.put(GENERATOR.buildColumnId("number"), AirbyteProtocolType.NUMBER); + COLUMNS.put(GENERATOR.buildColumnId("integer"), AirbyteProtocolType.INTEGER); + COLUMNS.put(GENERATOR.buildColumnId("boolean"), AirbyteProtocolType.BOOLEAN); + COLUMNS.put(GENERATOR.buildColumnId("timestamp_with_timezone"), AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE); + COLUMNS.put(GENERATOR.buildColumnId("timestamp_without_timezone"), AirbyteProtocolType.TIMESTAMP_WITHOUT_TIMEZONE); + COLUMNS.put(GENERATOR.buildColumnId("time_with_timezone"), AirbyteProtocolType.TIME_WITH_TIMEZONE); + COLUMNS.put(GENERATOR.buildColumnId("time_without_timezone"), AirbyteProtocolType.TIME_WITHOUT_TIMEZONE); + COLUMNS.put(GENERATOR.buildColumnId("date"), AirbyteProtocolType.DATE); + COLUMNS.put(GENERATOR.buildColumnId("unknown"), AirbyteProtocolType.UNKNOWN); + + CDC_COLUMNS = new LinkedHashMap<>(); + CDC_COLUMNS.put(ID_COLUMN, AirbyteProtocolType.INTEGER); + CDC_COLUMNS.put(CDC_CURSOR, AirbyteProtocolType.INTEGER); + CDC_COLUMNS.put(GENERATOR.buildColumnId("_ab_cdc_deleted_at"), AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE); + CDC_COLUMNS.put(GENERATOR.buildColumnId("struct"), new Struct(new LinkedHashMap<>())); + CDC_COLUMNS.put(GENERATOR.buildColumnId("array"), new Array(AirbyteProtocolType.UNKNOWN)); + CDC_COLUMNS.put(GENERATOR.buildColumnId("string"), AirbyteProtocolType.STRING); + CDC_COLUMNS.put(GENERATOR.buildColumnId("number"), AirbyteProtocolType.NUMBER); + CDC_COLUMNS.put(GENERATOR.buildColumnId("integer"), AirbyteProtocolType.INTEGER); + CDC_COLUMNS.put(GENERATOR.buildColumnId("boolean"), AirbyteProtocolType.BOOLEAN); + CDC_COLUMNS.put(GENERATOR.buildColumnId("timestamp_with_timezone"), AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE); + CDC_COLUMNS.put(GENERATOR.buildColumnId("timestamp_without_timezone"), AirbyteProtocolType.TIMESTAMP_WITHOUT_TIMEZONE); + CDC_COLUMNS.put(GENERATOR.buildColumnId("time_with_timezone"), AirbyteProtocolType.TIME_WITH_TIMEZONE); + CDC_COLUMNS.put(GENERATOR.buildColumnId("time_without_timezone"), AirbyteProtocolType.TIME_WITHOUT_TIMEZONE); + CDC_COLUMNS.put(GENERATOR.buildColumnId("date"), AirbyteProtocolType.DATE); + CDC_COLUMNS.put(GENERATOR.buildColumnId("unknown"), AirbyteProtocolType.UNKNOWN); + } + + @BeforeAll + public static void setup() throws Exception { + String rawConfig = Files.readString(Path.of("secrets/credentials-gcs-staging.json")); + JsonNode config = Jsons.deserialize(rawConfig); + + bq = BigQueryDestination.getBigQuery(config); + } + + @BeforeEach + public void setupDataset() { + testDataset = "bq_sql_generator_test_" + UUID.randomUUID().toString().replace("-", "_"); + // This is not a typical stream ID would look like, but we're just using this to isolate our tests + // to a specific dataset. + // In practice, the final table would be testDataset.users, and the raw table would be + // airbyte.testDataset_users. + streamId = new StreamId(testDataset, "users_final", testDataset, "users_raw", testDataset, "users_final"); + LOGGER.info("Running in dataset {}", testDataset); + + bq.create(DatasetInfo.newBuilder(testDataset) + // This unfortunately doesn't delete the actual dataset after 3 days, but at least we can clear out + // the tables if the AfterEach is skipped. + .setDefaultTableLifetime(Duration.ofDays(3).toMillis()) + .build()); + } + + @AfterEach + public void teardownDataset() { + bq.delete(testDataset, BigQuery.DatasetDeleteOption.deleteContents()); + } + + @Test + public void testCreateTableIncremental() throws InterruptedException { + StreamConfig stream = incrementalDedupStreamConfig(); + + logAndExecute(GENERATOR.createTable(stream, "")); + + final Table table = bq.getTable(testDataset, "users_final"); + // The table should exist + assertNotNull(table); + final Schema schema = table.getDefinition().getSchema(); + // And we should know exactly what columns it contains + 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(StandardSQLTypeName.STRING)).setMode(Mode.REQUIRED).build(), + Field.newBuilder("_airbyte_extracted_at", legacySQLTypeName(StandardSQLTypeName.TIMESTAMP)).setMode(Mode.REQUIRED).build(), + Field.newBuilder("_airbyte_meta", legacySQLTypeName(StandardSQLTypeName.JSON)).setMode(Mode.REQUIRED).build(), + Field.of("id", legacySQLTypeName(StandardSQLTypeName.INT64)), + Field.of("updated_at", legacySQLTypeName(StandardSQLTypeName.TIMESTAMP)), + Field.of("struct", legacySQLTypeName(StandardSQLTypeName.JSON)), + Field.of("array", legacySQLTypeName(StandardSQLTypeName.JSON)), + Field.of("string", legacySQLTypeName(StandardSQLTypeName.STRING)), + Field.of("number", legacySQLTypeName(StandardSQLTypeName.NUMERIC)), + Field.of("integer", legacySQLTypeName(StandardSQLTypeName.INT64)), + Field.of("boolean", legacySQLTypeName(StandardSQLTypeName.BOOL)), + Field.of("timestamp_with_timezone", legacySQLTypeName(StandardSQLTypeName.TIMESTAMP)), + Field.of("timestamp_without_timezone", legacySQLTypeName(StandardSQLTypeName.DATETIME)), + Field.of("time_with_timezone", legacySQLTypeName(StandardSQLTypeName.STRING)), + Field.of("time_without_timezone", legacySQLTypeName(StandardSQLTypeName.TIME)), + Field.of("date", legacySQLTypeName(StandardSQLTypeName.DATE)), + Field.of("unknown", legacySQLTypeName(StandardSQLTypeName.JSON))), + schema); + // TODO this should assert partitioning/clustering configs + } + + @Test + public void testVerifyPrimaryKeysIncremental() throws InterruptedException { + createRawTable(); + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`) VALUES + (JSON'{}', '10d6e27d-ae7a-41b5-baf8-c4c277ef9c11', '2023-01-01T00:00:00Z'), + (JSON'{"id": 1}', '5ce60e70-98aa-4fe3-8159-67207352c4f0', '2023-01-01T00:00:00Z'); + """)) + .build()); + + // This variable is declared outside of the transaction, so we need to do it manually here + final String sql = "DECLARE missing_pk_count INT64;" + GENERATOR.validatePrimaryKeys(streamId, List.of(new ColumnId("id", "id", "id")), COLUMNS); + final BigQueryException e = assertThrows( + BigQueryException.class, + () -> logAndExecute(sql)); + + assertTrue(e.getError().getMessage().startsWith("Raw table has 1 rows missing a primary key at"), + "Message was actually: " + e.getError().getMessage()); + } + + @Test + public void testInsertNewRecordsIncremental() throws InterruptedException { + createRawTable(); + createFinalTable(); + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`) VALUES + (JSON'{"id": 1, "updated_at": "2023-01-01T01:00:00Z", "string": "Alice", "struct": {"city": "San Francisco", "state": "CA"}}', '972fa08a-aa06-4b91-a6af-a371aee4cb1c', '2023-01-01T00:00:00Z'), + (JSON'{"id": 1, "updated_at": "2023-01-01T02:00:00Z", "string": "Alice", "struct": {"city": "San Diego", "state": "CA"}}', '233ad43d-de50-4a47-bbe6-7a417ce60d9d', '2023-01-01T00:00:00Z'), + (JSON'{"id": 2, "updated_at": "2023-01-01T03:00:00Z", "string": "Bob", "integer": "oops"}', 'd4aeb036-2d95-4880-acd2-dc69b42b03c6', '2023-01-01T00:00:00Z'); + """)) + .build()); + + final String sql = GENERATOR.insertNewRecords(streamId, "", COLUMNS, DestinationSyncMode.OVERWRITE); + logAndExecute(sql); + + final TableResult result = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.finalTableId(QUOTE)).build()); + DIFFER.diffFinalTableRecords( + List.of( + Jsons.deserialize( + """ + { + "id": 1, + "updated_at": "2023-01-01T01:00:00Z", + "string": "Alice", + "struct": {"city": "San Francisco", "state": "CA"}, + "_airbyte_extracted_at": "2023-01-01T00:00:00Z", + "_airbyte_meta": {"errors":[]} + } + """ + ), + Jsons.deserialize( + """ + { + "id": 1, + "updated_at": "2023-01-01T02:00:00Z", + "string": "Alice", + "struct": {"city": "San Diego", "state": "CA"}, + "_airbyte_extracted_at": "2023-01-01T00:00:00Z", + "_airbyte_meta": {"errors":[]} + } + """ + ), + Jsons.deserialize( + """ + { + "id": 2, + "updated_at": "2023-01-01T03:00:00Z", + "string": "Bob", + "_airbyte_extracted_at": "2023-01-01T00:00:00Z", + "_airbyte_meta": {"errors":["Problem with `integer`"]} + } + """ + )), + toJsonRecords(result)); + } + + @Test + public void testDedupFinalTable() throws InterruptedException { + createRawTable(); + createFinalTable(); + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`) VALUES + (JSON'{"id": 1, "updated_at": "2023-01-01T01:00:00Z", "string": "Alice", "struct": {"city": "San Francisco", "state": "CA"}, "integer": 42}', 'd7b81af0-01da-4846-a650-cc398986bc99', '2023-01-01T00:00:00Z'), + (JSON'{"id": 1, "updated_at": "2023-01-01T02:00:00Z", "string": "Alice", "struct": {"city": "San Diego", "state": "CA"}, "integer": 84}', '80c99b54-54b4-43bd-b51b-1f67dafa2c52', '2023-01-01T00:00:00Z'), + (JSON'{"id": 2, "updated_at": "2023-01-01T03:00:00Z", "string": "Bob", "integer": "oops"}', 'ad690bfb-c2c2-4172-bd73-a16c86ccbb67', '2023-01-01T00:00:00Z'); + + INSERT INTO ${dataset}.users_final (_airbyte_raw_id, _airbyte_extracted_at, _airbyte_meta, `id`, `updated_at`, `string`, `struct`, `integer`) values + ('d7b81af0-01da-4846-a650-cc398986bc99', '2023-01-01T00:00:00Z', JSON'{"errors":[]}', 1, '2023-01-01T01:00:00Z', 'Alice', JSON'{"city": "San Francisco", "state": "CA"}', 42), + ('80c99b54-54b4-43bd-b51b-1f67dafa2c52', '2023-01-01T00:00:00Z', JSON'{"errors":[]}', 1, '2023-01-01T02:00:00Z', 'Alice', JSON'{"city": "San Diego", "state": "CA"}', 84), + ('ad690bfb-c2c2-4172-bd73-a16c86ccbb67', '2023-01-01T00:00:00Z', JSON'{"errors": ["blah blah integer"]}', 2, '2023-01-01T03:00:00Z', 'Bob', NULL, NULL); + """)) + .build()); + + final String sql = GENERATOR.dedupFinalTable(streamId, "", PRIMARY_KEY, CURSOR); + logAndExecute(sql); + + final TableResult result = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.finalTableId(QUOTE)).build()); + DIFFER.diffFinalTableRecords( + List.of( + Jsons.deserialize( + """ + { + "id": 1, + "updated_at": "2023-01-01T02:00:00Z", + "string": "Alice", + "struct": {"city": "San Diego", "state": "CA"}, + "integer": 84, + "_airbyte_extracted_at": "2023-01-01T00:00:00Z", + "_airbyte_meta": {"errors":[]} + } + """), + Jsons.deserialize( + """ + { + "id": 2, + "updated_at": "2023-01-01T03:00:00Z", + "string": "Bob", + "_airbyte_extracted_at": "2023-01-01T00:00:00Z", + "_airbyte_meta": {"errors":["blah blah integer"]} + } + """)), + toJsonRecords(result)); + } + + @Test + public void testDedupRawTable() throws InterruptedException { + createRawTable(); + createFinalTable(); + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`) VALUES + (JSON'{"id": 1, "updated_at": "2023-01-01T01:00:00Z", "string": "Alice", "struct": {"city": "San Francisco", "state": "CA"}, "integer": 42}', 'd7b81af0-01da-4846-a650-cc398986bc99', '2023-01-01T00:00:00Z'), + (JSON'{"id": 1, "updated_at": "2023-01-01T02:00:00Z", "string": "Alice", "struct": {"city": "San Diego", "state": "CA"}, "integer": 84}', '80c99b54-54b4-43bd-b51b-1f67dafa2c52', '2023-01-01T00:00:00Z'), + (JSON'{"id": 2, "updated_at": "2023-01-01T03:00:00Z", "string": "Bob", "integer": "oops"}', 'ad690bfb-c2c2-4172-bd73-a16c86ccbb67', '2023-01-01T00:00:00Z'); + + INSERT INTO ${dataset}.users_final (_airbyte_raw_id, _airbyte_extracted_at, _airbyte_meta, `id`, `updated_at`, `string`, `struct`, `integer`) values + ('80c99b54-54b4-43bd-b51b-1f67dafa2c52', '2023-01-01T00:00:00Z', JSON'{"errors":[]}', 1, '2023-01-01T02:00:00Z', 'Alice', JSON'{"city": "San Diego", "state": "CA"}', 84), + ('ad690bfb-c2c2-4172-bd73-a16c86ccbb67', '2023-01-01T00:00:00Z', JSON'{"errors": ["blah blah integer"]}', 2, '2023-01-01T03:00:00Z', 'Bob', NULL, NULL); + """)) + .build()); + + final String sql = GENERATOR.dedupRawTable(streamId, ""); + logAndExecute(sql); + + final TableResult result = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.rawTableId(QUOTE)).build()); + DIFFER.diffRawTableRecords( + List.of( + Jsons.deserialize( + """ + { + "_airbyte_raw_id": "80c99b54-54b4-43bd-b51b-1f67dafa2c52", + "_airbyte_extracted_at": "2023-01-01T00:00:00Z", + "_airbyte_data": {"id": 1, "updated_at": "2023-01-01T02:00:00Z", "string": "Alice", "struct": {"city": "San Diego", "state": "CA"}, "integer": 84} + } + """ + ), + Jsons.deserialize( + """ + { + "_airbyte_raw_id": "ad690bfb-c2c2-4172-bd73-a16c86ccbb67", + "_airbyte_extracted_at": "2023-01-01T00:00:00Z", + "_airbyte_data": {"id": 2, "updated_at": "2023-01-01T03:00:00Z", "string": "Bob", "integer": "oops"} + } + """ + )), + toJsonRecords(result)); + } + + @Test + public void testCommitRawTable() throws InterruptedException { + createRawTable(); + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`) VALUES + (JSON'{"id": 1, "updated_at": "2023-01-01T02:00:00Z", "string": "Alice", "struct": {"city": "San Diego", "state": "CA"}, "integer": 84}', '80c99b54-54b4-43bd-b51b-1f67dafa2c52', '2023-01-01T00:00:00Z'), + (JSON'{"id": 2, "updated_at": "2023-01-01T03:00:00Z", "string": "Bob", "integer": "oops"}', 'ad690bfb-c2c2-4172-bd73-a16c86ccbb67', '2023-01-01T00:00:00Z'); + """)) + .build()); + + final String sql = GENERATOR.commitRawTable(streamId); + logAndExecute(sql); + + final long rawUntypedRows = bq.query(QueryJobConfiguration.newBuilder( + "SELECT * FROM " + streamId.rawTableId(QUOTE) + " WHERE _airbyte_loaded_at IS NULL").build()).getTotalRows(); + assertEquals(0, rawUntypedRows); + } + + @Test + public void testFullUpdateAllTypes() throws InterruptedException { + createRawTable(); + createFinalTable("_foo"); + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + INSERT INTO ${dataset}.users_raw (`_airbyte_raw_id`, `_airbyte_extracted_at`, `_airbyte_data`) VALUES + (generate_uuid(), '2023-01-01T00:00:00Z', JSON'{"id": 1, "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": {}}'), + (generate_uuid(), '2023-01-01T00:00:00Z', JSON'{"id": 2, "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}'), + (generate_uuid(), '2023-01-01T00:00:00Z', JSON'{"id": 3, "updated_at": "2023-01-01T01:00:00Z"}'), + (generate_uuid(), '2023-01-01T00:00:00Z', JSON'{"id": 4, "updated_at": "2023-01-01T01:00:00Z", "array": {}, "struct": [], "string": {}, "number": {}, "integer": {}, "boolean": {}, "timestamp_with_timezone": {}, "timestamp_without_timezone": {}, "time_with_timezone": {}, "time_without_timezone": {}, "date": {}, "unknown": null}'); + """)) + .build()); + + final String sql = GENERATOR.updateTable("_foo", incrementalDedupStreamConfig()); + logAndExecute(sql); + + final TableResult finalTable = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.finalTableId("_foo", QUOTE)).build()); + DIFFER.diffFinalTableRecords( + List.of( + Jsons.deserialize( + """ + { + "id": 1, + "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": {"errors": []} + } + """), + Jsons.deserialize( + """ + { + "id": 2, + "updated_at": "2023-01-01T01:00:00Z", + "unknown": null, + "_airbyte_extracted_at": "2023-01-01T00:00:00Z", + "_airbyte_meta": {"errors": []} + } + """), + Jsons.deserialize( + """ + { + "id": 3, + "updated_at": "2023-01-01T01:00:00Z", + "_airbyte_extracted_at": "2023-01-01T00:00:00Z", + "_airbyte_meta": {"errors": []} + } + """), + Jsons.deserialize( + """ + { + "id": 4, + "updated_at": "2023-01-01T01:00:00Z", + "unknown": null, + "_airbyte_extracted_at": "2023-01-01T00:00:00Z", + "_airbyte_meta": { + "errors": [ + "Problem with `struct`", + "Problem with `array`", + "Problem with `string`", + "Problem with `number`", + "Problem with `integer`", + "Problem with `boolean`", + "Problem with `timestamp_with_timezone`", + "Problem with `timestamp_without_timezone`", + "Problem with `time_with_timezone`", + "Problem with `time_without_timezone`", + "Problem with `date`" + ] + } + } + """)), + toJsonRecords(finalTable)); + + final long rawRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.rawTableId(QUOTE)).build()).getTotalRows(); + assertEquals(4, rawRows); + final long rawUntypedRows = bq.query(QueryJobConfiguration.newBuilder( + "SELECT * FROM " + streamId.rawTableId(QUOTE) + " WHERE _airbyte_loaded_at IS NULL").build()).getTotalRows(); + assertEquals(0, rawUntypedRows); + } + + @Test + public void testFullUpdateIncrementalDedup() throws InterruptedException { + createRawTable(); + createFinalTable(); + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`) VALUES + (JSON'{"id": 1, "updated_at": "2023-01-01T01:00:00Z", "string": "Alice", "struct": {"city": "San Francisco", "state": "CA"}, "integer": 42}', 'd7b81af0-01da-4846-a650-cc398986bc99', '2023-01-01T00:00:00Z'), + (JSON'{"id": 1, "updated_at": "2023-01-01T02:00:00Z", "string": "Alice", "struct": {"city": "San Diego", "state": "CA"}, "integer": 84}', '80c99b54-54b4-43bd-b51b-1f67dafa2c52', '2023-01-01T00:00:00Z'), + (JSON'{"id": 2, "updated_at": "2023-01-01T03:00:00Z", "string": "Bob", "integer": "oops"}', 'ad690bfb-c2c2-4172-bd73-a16c86ccbb67', '2023-01-01T00:00:00Z'); + """)) + .build()); + + final String sql = GENERATOR.updateTable("", incrementalDedupStreamConfig()); + logAndExecute(sql); + + // TODO + final long finalRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.finalTableId(QUOTE)).build()).getTotalRows(); + assertEquals(2, finalRows); + final long rawRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.rawTableId(QUOTE)).build()).getTotalRows(); + assertEquals(2, rawRows); + final long rawUntypedRows = bq.query(QueryJobConfiguration.newBuilder( + "SELECT * FROM " + streamId.rawTableId(QUOTE) + " WHERE _airbyte_loaded_at IS NULL").build()).getTotalRows(); + assertEquals(0, rawUntypedRows); + } + + @Test + public void testFullUpdateIncrementalAppend() throws InterruptedException { + createRawTable(); + createFinalTable("_foo"); + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`) VALUES + (JSON'{"id": 1, "updated_at": "2023-01-01T01:00:00Z", "string": "Alice", "struct": {"city": "San Francisco", "state": "CA"}, "integer": 42}', 'd7b81af0-01da-4846-a650-cc398986bc99', '2023-01-01T00:00:00Z'), + (JSON'{"id": 1, "updated_at": "2023-01-01T02:00:00Z", "string": "Alice", "struct": {"city": "San Diego", "state": "CA"}, "integer": 84}', '80c99b54-54b4-43bd-b51b-1f67dafa2c52', '2023-01-01T00:00:00Z'), + (JSON'{"id": 2, "updated_at": "2023-01-01T03:00:00Z", "string": "Bob", "integer": "oops"}', 'ad690bfb-c2c2-4172-bd73-a16c86ccbb67', '2023-01-01T00:00:00Z'); + """)) + .build()); + + final String sql = GENERATOR.updateTable("_foo", incrementalAppendStreamConfig()); + logAndExecute(sql); + + // TODO + final long finalRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.finalTableId("_foo", QUOTE)).build()).getTotalRows(); + assertEquals(3, finalRows); + final long rawRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.rawTableId(QUOTE)).build()).getTotalRows(); + assertEquals(3, rawRows); + final long rawUntypedRows = bq.query(QueryJobConfiguration.newBuilder( + "SELECT * FROM " + streamId.rawTableId(QUOTE) + " WHERE _airbyte_loaded_at IS NULL").build()).getTotalRows(); + assertEquals(0, rawUntypedRows); + } + + // This is also effectively the full refresh overwrite test case. + // In the overwrite case, we rely on the destination connector to tell us to write to a final table + // with a _tmp suffix, and then call overwriteFinalTable at the end of the sync. + @Test + public void testFullUpdateFullRefreshAppend() throws InterruptedException { + createRawTable(); + createFinalTable("_foo"); + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`) VALUES + (JSON'{"id": 1, "updated_at": "2023-01-01T01:00:00Z", "string": "Alice", "struct": {"city": "San Francisco", "state": "CA"}, "integer": 42}', 'd7b81af0-01da-4846-a650-cc398986bc99', '2023-01-01T00:00:00Z'), + (JSON'{"id": 1, "updated_at": "2023-01-01T02:00:00Z", "string": "Alice", "struct": {"city": "San Diego", "state": "CA"}, "integer": 84}', '80c99b54-54b4-43bd-b51b-1f67dafa2c52', '2023-01-01T00:00:00Z'), + (JSON'{"id": 2, "updated_at": "2023-01-01T03:00:00Z", "string": "Bob", "integer": "oops"}', 'ad690bfb-c2c2-4172-bd73-a16c86ccbb67', '2023-01-01T00:00:00Z'); + + INSERT INTO ${dataset}.users_final_foo (_airbyte_raw_id, _airbyte_extracted_at, _airbyte_meta, `id`, `updated_at`, `string`, `struct`, `integer`) values + ('64f4390f-3da1-4b65-b64a-a6c67497f18d', '2022-12-31T00:00:00Z', JSON'{"errors": []}', 1, '2022-12-31T00:00:00Z', 'Alice', NULL, NULL); + """)) + .build()); + + final String sql = GENERATOR.updateTable("_foo", fullRefreshAppendStreamConfig()); + logAndExecute(sql); + + // TODO + final long finalRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.finalTableId("_foo", QUOTE)).build()).getTotalRows(); + assertEquals(4, finalRows); + final long rawRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.rawTableId(QUOTE)).build()).getTotalRows(); + assertEquals(3, rawRows); + final long rawUntypedRows = bq.query(QueryJobConfiguration.newBuilder( + "SELECT * FROM " + streamId.rawTableId(QUOTE) + " WHERE _airbyte_loaded_at IS NULL").build()).getTotalRows(); + assertEquals(0, rawUntypedRows); + } + + @Test + public void testRenameFinalTable() throws InterruptedException { + createFinalTable("_tmp"); + + final String sql = GENERATOR.overwriteFinalTable("_tmp", fullRefreshOverwriteStreamConfig()).get(); + logAndExecute(sql); + + final Table table = bq.getTable(testDataset, "users_final"); + // TODO this should assert table schema + partitioning/clustering configs + assertNotNull(table); + } + + @Test + public void testCdcBasics() throws InterruptedException { + createRawTable(); + createFinalTableCdc(); + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`, `_airbyte_loaded_at`) VALUES + (JSON'{"id": 1, "_ab_cdc_lsn": 10001, "_ab_cdc_deleted_at": "2023-01-01T00:01:00Z"}', generate_uuid(), '2023-01-01T00:00:00Z', NULL); + """)) + .build()); + + final String sql = GENERATOR.updateTable("", cdcStreamConfig()); + logAndExecute(sql); + + // TODO better asserts + final long finalRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.finalTableId("", QUOTE)).build()).getTotalRows(); + assertEquals(0, finalRows); + final long rawRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.rawTableId(QUOTE)).build()).getTotalRows(); + assertEquals(1, rawRows); + final long rawUntypedRows = bq.query(QueryJobConfiguration.newBuilder( + "SELECT * FROM " + streamId.rawTableId(QUOTE) + " WHERE _airbyte_loaded_at IS NULL").build()).getTotalRows(); + assertEquals(0, rawUntypedRows); + } + + @Test + public void testCdcUpdate() throws InterruptedException { + createRawTable(); + createFinalTableCdc(); + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + -- records from a previous sync + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`, `_airbyte_loaded_at`) VALUES + (JSON'{"id": 1, "_ab_cdc_lsn": 900, "string": "spooky ghost", "_ab_cdc_deleted_at": null}', '64f4390f-3da1-4b65-b64a-a6c67497f18d', '2022-12-31T00:00:00Z', '2022-12-31T00:00:01Z'), + (JSON'{"id": 0, "_ab_cdc_lsn": 901, "string": "zombie", "_ab_cdc_deleted_at": "2022-12-31T00:O0:00Z"}', generate_uuid(), '2022-12-31T00:00:00Z', '2022-12-31T00:00:01Z'), + (JSON'{"id": 5, "_ab_cdc_lsn": 902, "string": "will not be deleted", "_ab_cdc_deleted_at": null}', 'b6139181-a42c-45c3-89f2-c4b4bb3a8c9d', '2022-12-31T00:00:00Z', '2022-12-31T00:00:01Z'); + INSERT INTO ${dataset}.users_final (_airbyte_raw_id, _airbyte_extracted_at, _airbyte_meta, `id`, `_ab_cdc_lsn`, `string`, `struct`, `integer`) values + ('64f4390f-3da1-4b65-b64a-a6c67497f18d', '2022-12-31T00:00:00Z', JSON'{}', 1, 900, 'spooky ghost', NULL, NULL), + ('b6139181-a42c-45c3-89f2-c4b4bb3a8c9d', '2022-12-31T00:00:00Z', JSON'{}', 5, 901, 'will be deleted', NULL, NULL); + + -- new records from the current sync + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`) VALUES + (JSON'{"id": 2, "_ab_cdc_lsn": 10001, "_ab_cdc_deleted_at": null, "string": "alice"}', generate_uuid(), '2023-01-01T00:00:00Z'), + (JSON'{"id": 2, "_ab_cdc_lsn": 10002, "_ab_cdc_deleted_at": null, "string": "alice2"}', generate_uuid(), '2023-01-01T00:00:00Z'), + (JSON'{"id": 3, "_ab_cdc_lsn": 10003, "_ab_cdc_deleted_at": null, "string": "bob"}', generate_uuid(), '2023-01-01T00:00:00Z'), + (JSON'{"id": 1, "_ab_cdc_lsn": 10004, "_ab_cdc_deleted_at": "2022-12-31T23:59:59Z"}', generate_uuid(), '2023-01-01T00:00:00Z'), + (JSON'{"id": 0, "_ab_cdc_lsn": 10005, "_ab_cdc_deleted_at": null, "string": "zombie_returned"}', generate_uuid(), '2023-01-01T00:00:00Z'), + -- CDC generally outputs an explicit null for deleted_at, but verify that we can also handle the case where deleted_at is unset. + (JSON'{"id": 4, "_ab_cdc_lsn": 10006, "string": "charlie"}', generate_uuid(), '2023-01-01T00:00:00Z'), + -- Verify that we can handle weird values in deleted_at + (JSON'{"id": 5, "_ab_cdc_lsn": 10007, "_ab_cdc_deleted_at": {}, "string": "david"}', generate_uuid(), '2023-01-01T00:00:00Z'); + """)) + .build()); + + final String sql = GENERATOR.updateTable("", cdcStreamConfig()); + logAndExecute(sql); + + final long finalRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.finalTableId("", QUOTE)).build()).getTotalRows(); + assertEquals(5, finalRows); + final long rawRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.rawTableId(QUOTE)).build()).getTotalRows(); + assertEquals(6, rawRows); // we only keep the newest raw record for reach PK + final long rawUntypedRows = bq.query(QueryJobConfiguration.newBuilder( + "SELECT * FROM " + streamId.rawTableId(QUOTE) + " WHERE _airbyte_loaded_at IS NULL").build()).getTotalRows(); + assertEquals(0, rawUntypedRows); + } + + /** + * source operations: + *

    + *
  1. insert id=1 (lsn 10000)
  2. + *
  3. delete id=1 (lsn 10001)
  4. + *
+ *

+ * But the destination writes lsn 10001 before 10000. We should still end up with no records in the + * final table. + *

+ * All records have the same emitted_at timestamp. This means that we live or die purely based on + * our ability to use _ab_cdc_lsn. + */ + @Test + public void testCdcOrdering_updateAfterDelete() throws InterruptedException { + createRawTable(); + createFinalTableCdc(); + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + -- Write raw deletion record from the first batch, which resulted in an empty final table. + -- Note the non-null loaded_at - this is to simulate that we previously ran T+D on this record. + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`, `_airbyte_loaded_at`) VALUES + (JSON'{"id": 1, "_ab_cdc_lsn": 10001, "_ab_cdc_deleted_at": "2023-01-01T00:01:00Z"}', generate_uuid(), '2023-01-01T00:00:00Z', '2023-01-01T00:00:01Z'); + + -- insert raw record from the second record batch - this is an outdated record that should be ignored. + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`) VALUES + (JSON'{"id": 1, "_ab_cdc_lsn": 10000, "string": "alice"}', generate_uuid(), '2023-01-01T00:00:00Z'); + """)) + .build()); + + final String sql = GENERATOR.updateTable("", cdcStreamConfig()); + logAndExecute(sql); + + final long finalRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.finalTableId("", QUOTE)).build()).getTotalRows(); + assertEquals(0, finalRows); + final long rawRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.rawTableId(QUOTE)).build()).getTotalRows(); + assertEquals(1, rawRows); + final long rawUntypedRows = bq.query(QueryJobConfiguration.newBuilder( + "SELECT * FROM " + streamId.rawTableId(QUOTE) + " WHERE _airbyte_loaded_at IS NULL").build()).getTotalRows(); + assertEquals(0, rawUntypedRows); + } + + /** + * source operations: + *

    + *
  1. arbitrary history...
  2. + *
  3. delete id=1 (lsn 10001)
  4. + *
  5. reinsert id=1 (lsn 10002)
  6. + *
+ *

+ * But the destination receives LSNs 10002 before 10001. In this case, we should keep the reinserted + * record in the final table. + *

+ * All records have the same emitted_at timestamp. This means that we live or die purely based on + * our ability to use _ab_cdc_lsn. + */ + @Test + public void testCdcOrdering_insertAfterDelete() throws InterruptedException { + createRawTable(); + createFinalTableCdc(); + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + -- records from the first batch + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`, `_airbyte_loaded_at`) VALUES + (JSON'{"id": 1, "_ab_cdc_lsn": 10002, "string": "alice_reinsert"}', '64f4390f-3da1-4b65-b64a-a6c67497f18d', '2023-01-01T00:00:00Z', '2023-01-01T00:00:01Z'); + INSERT INTO ${dataset}.users_final (_airbyte_raw_id, _airbyte_extracted_at, _airbyte_meta, `id`, `_ab_cdc_lsn`, `string`) values + ('64f4390f-3da1-4b65-b64a-a6c67497f18d', '2023-01-01T00:00:00Z', JSON'{}', 1, 10002, 'alice_reinsert'); + + -- second record batch + INSERT INTO ${dataset}.users_raw (`_airbyte_data`, `_airbyte_raw_id`, `_airbyte_extracted_at`) VALUES + (JSON'{"id": 1, "_ab_cdc_lsn": 10001, "_ab_cdc_deleted_at": "2023-01-01T00:01:00Z"}', generate_uuid(), '2023-01-01T00:00:00Z'); + """)) + .build()); + // Run the second round of typing and deduping. This should do nothing to the final table, because + // the delete is outdated. + final String sql = GENERATOR.updateTable("", cdcStreamConfig()); + logAndExecute(sql); + + final long finalRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.finalTableId("", QUOTE)).build()).getTotalRows(); + assertEquals(1, finalRows); + final long rawRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.rawTableId(QUOTE)).build()).getTotalRows(); + assertEquals(1, rawRows); + final long rawUntypedRows = bq.query(QueryJobConfiguration.newBuilder( + "SELECT * FROM " + streamId.rawTableId(QUOTE) + " WHERE _airbyte_loaded_at IS NULL").build()).getTotalRows(); + assertEquals(0, rawUntypedRows); + } + + private StreamConfig incrementalDedupStreamConfig() { + return new StreamConfig( + streamId, + SyncMode.INCREMENTAL, + DestinationSyncMode.APPEND_DEDUP, + PRIMARY_KEY, + Optional.of(CURSOR), + COLUMNS); + } + + private StreamConfig cdcStreamConfig() { + return new StreamConfig( + streamId, + SyncMode.INCREMENTAL, + DestinationSyncMode.APPEND_DEDUP, + PRIMARY_KEY, + // Much like the rest of this class - this is purely for test purposes. Real CDC cursors may not be + // exactly the same as this. + Optional.of(CDC_CURSOR), + CDC_COLUMNS); + } + + private StreamConfig incrementalAppendStreamConfig() { + return new StreamConfig( + streamId, + SyncMode.INCREMENTAL, + DestinationSyncMode.APPEND, + null, + Optional.of(CURSOR), + COLUMNS); + } + + private StreamConfig fullRefreshAppendStreamConfig() { + return new StreamConfig( + streamId, + SyncMode.FULL_REFRESH, + DestinationSyncMode.APPEND, + null, + Optional.empty(), + COLUMNS); + } + + private StreamConfig fullRefreshOverwriteStreamConfig() { + return new StreamConfig( + streamId, + SyncMode.FULL_REFRESH, + DestinationSyncMode.OVERWRITE, + null, + Optional.empty(), + COLUMNS); + } + + // These are known-good methods for doing stuff with bigquery. + // Some of them are identical to what the sql generator does, and that's intentional. + private void createRawTable() throws InterruptedException { + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + CREATE TABLE ${dataset}.users_raw ( + _airbyte_raw_id STRING NOT NULL, + _airbyte_data JSON NOT NULL, + _airbyte_extracted_at TIMESTAMP NOT NULL, + _airbyte_loaded_at TIMESTAMP + ) PARTITION BY ( + DATE_TRUNC(_airbyte_extracted_at, DAY) + ) CLUSTER BY _airbyte_loaded_at; + """)) + .build()); + } + + private void createFinalTable() throws InterruptedException { + createFinalTable(""); + } + + private void createFinalTable(String suffix) throws InterruptedException { + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset, + "suffix", suffix)).replace( + """ + CREATE TABLE ${dataset}.users_final${suffix} ( + _airbyte_raw_id STRING NOT NULL, + _airbyte_extracted_at TIMESTAMP NOT NULL, + _airbyte_meta JSON NOT NULL, + `id` INT64, + `updated_at` TIMESTAMP, + `struct` JSON, + `array` JSON, + `string` STRING, + `number` NUMERIC, + `integer` INT64, + `boolean` BOOL, + `timestamp_with_timezone` TIMESTAMP, + `timestamp_without_timezone` DATETIME, + `time_with_timezone` STRING, + `time_without_timezone` TIME, + `date` DATE, + `unknown` JSON + ) + PARTITION BY (DATE_TRUNC(_airbyte_extracted_at, DAY)) + CLUSTER BY id, _airbyte_extracted_at; + """)) + .build()); + } + + private void createFinalTableCdc() throws InterruptedException { + bq.query(QueryJobConfiguration.newBuilder( + new StringSubstitutor(Map.of( + "dataset", testDataset)).replace( + """ + CREATE TABLE ${dataset}.users_final ( + _airbyte_raw_id STRING NOT NULL, + _airbyte_extracted_at TIMESTAMP NOT NULL, + _airbyte_meta JSON NOT NULL, + `id` INT64, + `_ab_cdc_deleted_at` TIMESTAMP, + `_ab_cdc_lsn` INT64, + `struct` JSON, + `array` JSON, + `string` STRING, + `number` NUMERIC, + `integer` INT64, + `boolean` BOOL, + `timestamp_with_timezone` TIMESTAMP, + `timestamp_without_timezone` DATETIME, + `time_with_timezone` STRING, + `time_without_timezone` TIME, + `date` DATE, + `unknown` JSON + ) + PARTITION BY (DATE_TRUNC(_airbyte_extracted_at, DAY)) + CLUSTER BY id, _airbyte_extracted_at; + """)) + .build()); + } + + private static void logAndExecute(final String sql) throws InterruptedException { + LOGGER.info("Executing sql: {}", sql); + bq.query(QueryJobConfiguration.newBuilder(sql).build()); + } + + /** + * 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. + */ + public static List toJsonRecords(TableResult result) { + return result.streamAll().map(row -> toJson(result.getSchema(), 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 static JsonNode toJson(Schema schema, FieldValueList row) { + final ObjectNode json = (ObjectNode) Jsons.emptyObject(); + for (int i = 0; i < schema.getFields().size(); i++) { + final Field field = schema.getFields().get(i); + final FieldValue value = row.get(i); + JsonNode typedValue; + if (!value.isNull()) { + typedValue = switch (field.getType().getStandardType()) { + case BOOL -> Jsons.jsonNode(value.getBooleanValue()); + case INT64 -> Jsons.jsonNode(value.getLongValue()); + case FLOAT64 -> Jsons.jsonNode(value.getDoubleValue()); + case NUMERIC, BIGNUMERIC -> Jsons.jsonNode(value.getNumericValue()); + case STRING -> Jsons.jsonNode(value.getStringValue()); + // naively converting an Instant returns a DecimalNode with the unix epoch, so instead we manually stringify it + case TIMESTAMP -> Jsons.jsonNode(value.getTimestampInstant().toString()); + // value.getTimestampInstant() fails to parse these types + case DATE, DATETIME, TIME -> Jsons.jsonNode(value.getStringValue()); + // bigquery returns JSON columns as string; manually parse it into a JsonNode + case JSON -> Jsons.jsonNode(Jsons.deserialize(value.getStringValue())); + + // Default case for weird types (struct, array, geography, interval, bytes) + default -> Jsons.jsonNode(value.getStringValue()); + }; + json.set(field.getName(), typedValue); + } + } + return json; + } + +} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryStandardInsertsTypingDedupingTest.java b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryStandardInsertsTypingDedupingTest.java new file mode 100644 index 000000000000..be86379f2719 --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryStandardInsertsTypingDedupingTest.java @@ -0,0 +1,9 @@ +package io.airbyte.integrations.destination.bigquery.typing_deduping; + +public class BigQueryStandardInsertsTypingDedupingTest extends AbstractBigQueryTypingDedupingTest { + + @Override + public String getConfigPath() { + return "secrets/credentials-1s1t-standard.json"; + } +} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/test/java/io/airbyte/integrations/destination/bigquery/BigQueryRecordConsumerTest.java b/airbyte-integrations/connectors/destination-bigquery/src/test/java/io/airbyte/integrations/destination/bigquery/BigQueryRecordConsumerTest.java index 27511e993453..e67ee15225b4 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/test/java/io/airbyte/integrations/destination/bigquery/BigQueryRecordConsumerTest.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/test/java/io/airbyte/integrations/destination/bigquery/BigQueryRecordConsumerTest.java @@ -4,15 +4,25 @@ package io.airbyte.integrations.destination.bigquery; +import static org.mockito.Mockito.mock; + +import com.google.cloud.bigquery.BigQuery; +import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.base.DestinationConfig; import io.airbyte.integrations.base.FailureTrackingAirbyteMessageConsumer; +import io.airbyte.integrations.base.TypingAndDedupingFlag; +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser; +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler; +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator; import io.airbyte.integrations.destination.bigquery.uploader.AbstractBigQueryUploader; import io.airbyte.integrations.standardtest.destination.PerStreamStateMessageTest; import io.airbyte.protocol.models.v0.AirbyteMessage; import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair; +import java.util.Collections; import java.util.Map; import java.util.function.Consumer; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.InjectMocks; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; @@ -24,9 +34,22 @@ public class BigQueryRecordConsumerTest extends PerStreamStateMessageTest { @Mock private Consumer outputRecordCollector; - @InjectMocks private BigQueryRecordConsumer bigQueryRecordConsumer; + @BeforeEach + public void setup() { + DestinationConfig.initialize(Jsons.deserialize("{}")); + + bigQueryRecordConsumer = new BigQueryRecordConsumer( + mock(BigQuery.class), + uploaderMap, + outputRecordCollector, + "test-dataset-id", + mock(BigQuerySqlGenerator.class), + mock(BigQueryDestinationHandler.class), + new CatalogParser.ParsedCatalog(Collections.emptyList())); + } + @Override protected Consumer getMockedConsumer() { return outputRecordCollector; diff --git a/airbyte-integrations/connectors/destination-bigquery/src/test/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGeneratorTest.java b/airbyte-integrations/connectors/destination-bigquery/src/test/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGeneratorTest.java new file mode 100644 index 000000000000..fb57e622662f --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/test/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGeneratorTest.java @@ -0,0 +1,57 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.destination.bigquery.typing_deduping; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import com.google.cloud.bigquery.StandardSQLTypeName; +import com.google.common.collect.ImmutableList; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.AirbyteProtocolType; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.Array; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.OneOf; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.Struct; +import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType.UnsupportedOneOf; +import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import org.junit.jupiter.api.Test; + +public class BigQuerySqlGeneratorTest { + + private final BigQuerySqlGenerator generator = new BigQuerySqlGenerator(); + + @Test + public void testToDialectType() { + final Struct s = new Struct(new LinkedHashMap<>()); + final Array a = new Array(AirbyteProtocolType.BOOLEAN); + + assertEquals(StandardSQLTypeName.INT64, generator.toDialectType((AirbyteType) AirbyteProtocolType.INTEGER)); + assertEquals(StandardSQLTypeName.JSON, generator.toDialectType(s)); + assertEquals(StandardSQLTypeName.JSON, generator.toDialectType(a)); + assertEquals(StandardSQLTypeName.JSON, generator.toDialectType(new UnsupportedOneOf(new ArrayList<>()))); + + OneOf o = new OneOf(ImmutableList.of(s)); + assertEquals(StandardSQLTypeName.JSON, generator.toDialectType(o)); + o = new OneOf(ImmutableList.of(a)); + assertEquals(StandardSQLTypeName.JSON, generator.toDialectType(o)); + o = new OneOf(ImmutableList.of(AirbyteProtocolType.BOOLEAN, AirbyteProtocolType.NUMBER)); + assertEquals(StandardSQLTypeName.NUMERIC, generator.toDialectType(o)); + } + + @Test + public void testBuildColumnId() { + // Uninteresting names are unchanged + assertEquals( + new SqlGenerator.ColumnId("foo", "foo", "foo"), + generator.buildColumnId("foo")); + // Certain strings can't be the start of a column name, so we prepend an underscore + // Also, downcase the canonical name + assertEquals( + new SqlGenerator.ColumnId("__TABLE_foo_bar", "_TABLE_foo_bar", "__table_foo_bar"), + generator.buildColumnId("_TABLE_foo_bar")); + } + +} diff --git a/airbyte-integrations/connectors/destination-gcs/Dockerfile b/airbyte-integrations/connectors/destination-gcs/Dockerfile index ae2e60f375f1..943134018e79 100644 --- a/airbyte-integrations/connectors/destination-gcs/Dockerfile +++ b/airbyte-integrations/connectors/destination-gcs/Dockerfile @@ -24,5 +24,5 @@ ENV APPLICATION destination-gcs COPY --from=build /airbyte /airbyte -LABEL io.airbyte.version=0.4.2 +LABEL io.airbyte.version=0.4.3 LABEL io.airbyte.name=airbyte/destination-gcs diff --git a/airbyte-integrations/connectors/destination-gcs/build.gradle b/airbyte-integrations/connectors/destination-gcs/build.gradle index 3aa35bf0f4e1..fe7fa51f3eb9 100644 --- a/airbyte-integrations/connectors/destination-gcs/build.gradle +++ b/airbyte-integrations/connectors/destination-gcs/build.gradle @@ -45,3 +45,12 @@ dependencies { integrationTestJavaImplementation project(':airbyte-integrations:connectors:destination-gcs') integrationTestJavaImplementation project(':airbyte-connector-test-harnesses:acceptance-test-harness') } + + + + + + + + + diff --git a/airbyte-integrations/connectors/destination-gcs/metadata.yaml b/airbyte-integrations/connectors/destination-gcs/metadata.yaml index 8f6b38a17cba..cdaf2869904a 100644 --- a/airbyte-integrations/connectors/destination-gcs/metadata.yaml +++ b/airbyte-integrations/connectors/destination-gcs/metadata.yaml @@ -2,7 +2,7 @@ data: connectorSubtype: file connectorType: destination definitionId: ca8f6566-e555-4b40-943a-545bf123117a - dockerImageTag: 0.4.2 + dockerImageTag: 0.4.3 dockerRepository: airbyte/destination-gcs githubIssueLabel: destination-gcs icon: googlecloudstorage.svg diff --git a/airbyte-integrations/connectors/destination-gcs/src/main/java/io/airbyte/integrations/destination/gcs/util/GcsUtils.java b/airbyte-integrations/connectors/destination-gcs/src/main/java/io/airbyte/integrations/destination/gcs/util/GcsUtils.java index c8d22cb0ed4b..9ea223b318f6 100644 --- a/airbyte-integrations/connectors/destination-gcs/src/main/java/io/airbyte/integrations/destination/gcs/util/GcsUtils.java +++ b/airbyte-integrations/connectors/destination-gcs/src/main/java/io/airbyte/integrations/destination/gcs/util/GcsUtils.java @@ -5,6 +5,7 @@ package io.airbyte.integrations.destination.gcs.util; import io.airbyte.integrations.base.JavaBaseConstants; +import io.airbyte.integrations.base.TypingAndDedupingFlag; import io.airbyte.integrations.destination.s3.avro.AvroConstants; import javax.annotation.Nullable; import org.apache.avro.LogicalTypes; @@ -18,6 +19,7 @@ public class GcsUtils { private static final Logger LOGGER = LoggerFactory.getLogger(GcsUtils.class); private static final Schema UUID_SCHEMA = LogicalTypes.uuid().addToSchema(Schema.create(Schema.Type.STRING)); private static final Schema TIMESTAMP_MILLIS_SCHEMA = LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + private static final Schema NULLABLE_TIMESTAMP_MILLIS = SchemaBuilder.builder().unionOf().nullType().and().type(TIMESTAMP_MILLIS_SCHEMA).endUnion(); public static Schema getDefaultAvroSchema(final String name, @Nullable final String namespace, @@ -30,14 +32,25 @@ public static Schema getDefaultAvroSchema(final String name, if (stdNamespace != null) { builder = builder.namespace(stdNamespace); } + if (TypingAndDedupingFlag.isDestinationV2()) { + builder.namespace("airbyte"); + } SchemaBuilder.FieldAssembler assembler = builder.fields(); - - if (appendAirbyteFields) { - assembler = assembler.name(JavaBaseConstants.COLUMN_NAME_AB_ID).type(UUID_SCHEMA).noDefault(); - assembler = assembler.name(JavaBaseConstants.COLUMN_NAME_EMITTED_AT).type(TIMESTAMP_MILLIS_SCHEMA).noDefault(); + if (TypingAndDedupingFlag.isDestinationV2()) { + if (appendAirbyteFields) { + assembler = assembler.name(JavaBaseConstants.COLUMN_NAME_AB_RAW_ID).type(UUID_SCHEMA).noDefault(); + assembler = assembler.name(JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT).type(TIMESTAMP_MILLIS_SCHEMA).noDefault(); + assembler = assembler.name(JavaBaseConstants.COLUMN_NAME_AB_LOADED_AT).type(NULLABLE_TIMESTAMP_MILLIS).withDefault(null); + } + assembler = assembler.name(JavaBaseConstants.COLUMN_NAME_DATA).type().stringType().noDefault(); + } else { + if (appendAirbyteFields) { + assembler = assembler.name(JavaBaseConstants.COLUMN_NAME_AB_ID).type(UUID_SCHEMA).noDefault(); + assembler = assembler.name(JavaBaseConstants.COLUMN_NAME_EMITTED_AT).type(TIMESTAMP_MILLIS_SCHEMA).noDefault(); + } + assembler = assembler.name(JavaBaseConstants.COLUMN_NAME_DATA).type().stringType().noDefault(); } - assembler = assembler.name(JavaBaseConstants.COLUMN_NAME_DATA).type().stringType().noDefault(); return assembler.endRecord(); } diff --git a/airbyte-integrations/connectors/destination-gcs/src/test/java/io/airbyte/integrations/destination/gcs/avro/GcsAvroWriterTest.java b/airbyte-integrations/connectors/destination-gcs/src/test/java/io/airbyte/integrations/destination/gcs/avro/GcsAvroWriterTest.java index 52f5d68907e7..fe1b3b30eb13 100644 --- a/airbyte-integrations/connectors/destination-gcs/src/test/java/io/airbyte/integrations/destination/gcs/avro/GcsAvroWriterTest.java +++ b/airbyte-integrations/connectors/destination-gcs/src/test/java/io/airbyte/integrations/destination/gcs/avro/GcsAvroWriterTest.java @@ -11,6 +11,8 @@ import com.amazonaws.services.s3.AmazonS3; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; +import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.base.DestinationConfig; import io.airbyte.integrations.destination.gcs.GcsDestinationConfig; import io.airbyte.integrations.destination.gcs.credential.GcsHmacKeyCredentialConfig; import io.airbyte.integrations.destination.s3.avro.S3AvroFormatConfig; @@ -26,6 +28,8 @@ class GcsAvroWriterTest { @Test public void generatesCorrectObjectPath() throws IOException { + DestinationConfig.initialize(Jsons.deserialize("{}")); + final GcsAvroWriter writer = new GcsAvroWriter( new GcsDestinationConfig( "fake-bucket", diff --git a/buildSrc/src/main/groovy/airbyte-integration-test-java.gradle b/buildSrc/src/main/groovy/airbyte-integration-test-java.gradle index a6938bc96791..e650889c417c 100644 --- a/buildSrc/src/main/groovy/airbyte-integration-test-java.gradle +++ b/buildSrc/src/main/groovy/airbyte-integration-test-java.gradle @@ -53,6 +53,11 @@ class AirbyteIntegrationTestJavaPlugin implements Plugin { // This is needed to make the destination-snowflake tests succeed - https://github.com/snowflakedb/snowflake-jdbc/issues/589#issuecomment-983944767 jvmArgs = ["--add-opens=java.base/java.nio=ALL-UNNAMED"] + + systemProperties = [ + // Allow tests to set @Execution(ExecutionMode.CONCURRENT) + 'junit.jupiter.execution.parallel.enabled': 'true' + ] } // make sure we create the integrationTest task once in case a standard source test was already initialized diff --git a/docs/connector-development/debugging-docker.md b/docs/connector-development/debugging-docker.md index bacedbcc0aeb..3f707fc0d3d8 100644 --- a/docs/connector-development/debugging-docker.md +++ b/docs/connector-development/debugging-docker.md @@ -84,7 +84,7 @@ In the `docker-compose.debug.yaml` file you should see an entry for the `worker` Similar to the previous debugging example, we want to pass an environment variable to the `docker compose` command. This time we're setting the `DEBUG_CONTAINER_IMAGE` environment variable to the name of the container we're targeting. For our example that is `destination-postgres` so run the command: ```bash -DEBUG_CONTAINER_IMAGE="destination-postgres" VERSION="dev" docker compose -f docker-compose.yaml -f docker-compose.debug.yaml up +DEBUG_CONTAINER_IMAGE="destination-postgres:5005" VERSION="dev" docker compose -f docker-compose.yaml -f docker-compose.debug.yaml up ``` The `worker` container now has an environment variable `DEBUG_CONTAINER_IMAGE` with a value of `destination-postgres` which when it compares when it is spawning containers. If the container name matches the environment variable, it will set the `JAVA_TOOL_OPTIONS` environment variable in the container to diff --git a/docs/integrations/destinations/bigquery-denormalized.md b/docs/integrations/destinations/bigquery-denormalized.md index d91f98338911..89b7c5444567 100644 --- a/docs/integrations/destinations/bigquery-denormalized.md +++ b/docs/integrations/destinations/bigquery-denormalized.md @@ -8,6 +8,7 @@ See [destinations/bigquery](https://docs.airbyte.com/integrations/destinations/b | Version | Date | Pull Request | Subject | | :------ | :--------- | :--------------------------------------------------------- | :----------------------------------------------------------------------------------------------------------------------- | +| 1.5.2 | 2023-07-05 | [\#27936](https://github.com/airbytehq/airbyte/pull/27936) | Internal code change | | 1.5.1 | 2023-06-30 | [\#27891](https://github.com/airbytehq/airbyte/pull/27891) | Revert bugged update | | 1.5.0 | 2023-06-27 | [\#27781](https://github.com/airbytehq/airbyte/pull/27781) | License Update: Elv2 | | 1.4.1 | 2023-05-17 | [\#26213](https://github.com/airbytehq/airbyte/pull/26213) | Fix bug in parsing file buffer config count | diff --git a/docs/integrations/destinations/bigquery.md b/docs/integrations/destinations/bigquery.md index a5e54133221c..42eba9f68b05 100644 --- a/docs/integrations/destinations/bigquery.md +++ b/docs/integrations/destinations/bigquery.md @@ -135,6 +135,7 @@ Now that you have set up the BigQuery destination connector, check out the follo | Version | Date | Pull Request | Subject | | :------ | :--------- | :--------------------------------------------------------- | :----------------------------------------------------------------------------------------------------------------------- | +| 1.5.2 | 2023-07-05 | [\#27936](https://github.com/airbytehq/airbyte/pull/27936) | Internal scaffolding change for future development | | 1.5.1 | 2023-06-30 | [\#27891](https://github.com/airbytehq/airbyte/pull/27891) | Revert bugged update | | 1.5.0 | 2023-06-27 | [\#27781](https://github.com/airbytehq/airbyte/pull/27781) | License Update: Elv2 | | 1.4.6 | 2023-06-28 | [\#27268](https://github.com/airbytehq/airbyte/pull/27268) | Internal scaffolding change for future development | diff --git a/docs/integrations/destinations/gcs.md b/docs/integrations/destinations/gcs.md index 7dfab9e51d52..92b1b28dddd2 100644 --- a/docs/integrations/destinations/gcs.md +++ b/docs/integrations/destinations/gcs.md @@ -237,6 +237,7 @@ Under the hood, an Airbyte data stream in Json schema is first converted to an A | Version | Date | Pull Request | Subject | | :------ | :--------- | :--------------------------------------------------------- | :------------------------------------------------------------------------------------------------------------------------- | +| 0.4.3 | 2023-07-05 | [#27936](https://github.com/airbytehq/airbyte/pull/27936) | Internal code update | | 0.4.2 | 2023-06-30 | [#27891](https://github.com/airbytehq/airbyte/pull/27891) | Internal code update | | 0.4.1 | 2023-06-28 | [#27268](https://github.com/airbytehq/airbyte/pull/27268) | Internal code update | | 0.4.0 | 2023-06-26 | [#27725](https://github.com/airbytehq/airbyte/pull/27725) | License Update: Elv2 | diff --git a/docs/understanding-airbyte/supported-data-types.md b/docs/understanding-airbyte/supported-data-types.md index ca84b13759fd..cb11bf5de539 100644 --- a/docs/understanding-airbyte/supported-data-types.md +++ b/docs/understanding-airbyte/supported-data-types.md @@ -46,7 +46,7 @@ For example, a source could produce this `AirbyteStream` (remember that the `jso "items": { "type": "string", "format": "date-time", - "airbyte_type": "timestampt_with_timezone" + "airbyte_type": "timestamp_with_timezone" } } } diff --git a/settings.gradle b/settings.gradle index 54bee84c8a99..ecbf49bb4a6b 100644 --- a/settings.gradle +++ b/settings.gradle @@ -104,6 +104,8 @@ if (!System.getenv().containsKey("SUB_BUILD") || System.getenv().get("SUB_BUILD" include ':airbyte-integrations:bases:base-java' include ':airbyte-integrations:bases:base-java-s3' include ':airbyte-integrations:bases:base-normalization' + include ':airbyte-integrations:bases:base-typing-deduping' + include ':airbyte-integrations:bases:base-typing-deduping-test' include ':airbyte-integrations:bases:bases-destination-jdbc' // needs to be lexicographically after base-java and base-normalization to avoid race condition include ':airbyte-integrations:bases:base-standard-source-test-file' include ':airbyte-integrations:bases:connector-acceptance-test' diff --git a/tools/ci_connector_ops/ci_connector_ops/pipelines/actions/environments.py b/tools/ci_connector_ops/ci_connector_ops/pipelines/actions/environments.py index cb21fd46af3d..9b4b8b41a244 100644 --- a/tools/ci_connector_ops/ci_connector_ops/pipelines/actions/environments.py +++ b/tools/ci_connector_ops/ci_connector_ops/pipelines/actions/environments.py @@ -636,6 +636,7 @@ def with_integration_base_java(context: PipelineContext, build_platform: Platfor "dockerfile": "Dockerfile", "dbt_adapter": "dbt-bigquery==1.0.0", "integration_name": "bigquery", + "normalization_image": "airbyte/normalization:0.4.3", "supports_in_connector_normalization": True, "yum_packages": [], }, @@ -643,6 +644,7 @@ def with_integration_base_java(context: PipelineContext, build_platform: Platfor "dockerfile": "clickhouse.Dockerfile", "dbt_adapter": "dbt-clickhouse>=1.4.0", "integration_name": "clickhouse", + "normalization_image": "airbyte/normalization-clickhouse:0.4.3", "supports_in_connector_normalization": False, "yum_packages": [], }, @@ -650,6 +652,7 @@ def with_integration_base_java(context: PipelineContext, build_platform: Platfor "dockerfile": "duckdb.Dockerfile", "dbt_adapter": "dbt-duckdb==1.0.1", "integration_name": "duckdb", + "normalization_image": "airbyte/normalization-duckdb:0.4.3", "supports_in_connector_normalization": False, "yum_packages": [], }, @@ -657,6 +660,7 @@ def with_integration_base_java(context: PipelineContext, build_platform: Platfor "dockerfile": "mssql.Dockerfile", "dbt_adapter": "dbt-sqlserver==1.0.0", "integration_name": "mssql", + "normalization_image": "airbyte/normalization-mssql:0.4.3", "supports_in_connector_normalization": True, "yum_packages": [], }, @@ -664,6 +668,7 @@ def with_integration_base_java(context: PipelineContext, build_platform: Platfor "dockerfile": "mysql.Dockerfile", "dbt_adapter": "dbt-mysql==1.0.0", "integration_name": "mysql", + "normalization_image": "airbyte/normalization-mysql:0.4.3", "supports_in_connector_normalization": False, "yum_packages": [], }, @@ -671,6 +676,7 @@ def with_integration_base_java(context: PipelineContext, build_platform: Platfor "dockerfile": "oracle.Dockerfile", "dbt_adapter": "dbt-oracle==0.4.3", "integration_name": "oracle", + "normalization_image": "airbyte/normalization-oracle:0.4.3", "supports_in_connector_normalization": False, "yum_packages": [], }, @@ -678,6 +684,7 @@ def with_integration_base_java(context: PipelineContext, build_platform: Platfor "dockerfile": "Dockerfile", "dbt_adapter": "dbt-postgres==1.0.0", "integration_name": "postgres", + "normalization_image": "airbyte/normalization:0.4.3", "supports_in_connector_normalization": False, "yum_packages": [], }, @@ -685,6 +692,7 @@ def with_integration_base_java(context: PipelineContext, build_platform: Platfor "dockerfile": "redshift.Dockerfile", "dbt_adapter": "dbt-redshift==1.0.0", "integration_name": "redshift", + "normalization_image": "airbyte/normalization-redshift:0.4.3", "supports_in_connector_normalization": True, "yum_packages": [], }, @@ -692,6 +700,7 @@ def with_integration_base_java(context: PipelineContext, build_platform: Platfor "dockerfile": "snowflake.Dockerfile", "dbt_adapter": "dbt-snowflake==1.0.0", "integration_name": "snowflake", + "normalization_image": "airbyte/normalization-snowflake:0.4.3", "supports_in_connector_normalization": True, "yum_packages": ["gcc-c++"], }, @@ -699,6 +708,7 @@ def with_integration_base_java(context: PipelineContext, build_platform: Platfor "dockerfile": "tidb.Dockerfile", "dbt_adapter": "dbt-tidb==1.0.1", "integration_name": "tidb", + "normalization_image": "airbyte/normalization-tidb:0.4.3", "supports_in_connector_normalization": True, "yum_packages": [], }, @@ -710,15 +720,10 @@ def with_integration_base_java(context: PipelineContext, build_platform: Platfor } -def with_normalization(context: ConnectorContext) -> Container: - normalization_directory = context.get_repo_dir("airbyte-integrations/bases/base-normalization") - sshtunneling_file = context.get_repo_dir( - "airbyte-connector-test-harnesses/acceptance-test-harness/src/main/resources", include="sshtunneling.sh" - ).file("sshtunneling.sh") - normalization_directory_with_build = normalization_directory.with_new_directory("build") - normalization_directory_with_sshtunneling = normalization_directory_with_build.with_file("build/sshtunneling.sh", sshtunneling_file) - normalization_dockerfile_name = DESTINATION_NORMALIZATION_BUILD_CONFIGURATION[context.connector.technical_name]["dockerfile"] - return normalization_directory_with_sshtunneling.docker_build(normalization_dockerfile_name) +def with_normalization(context: ConnectorContext, build_platform: Platform) -> Container: + return context.dagger_client.container(platform=build_platform).from_( + DESTINATION_NORMALIZATION_BUILD_CONFIGURATION[context.connector.technical_name]["normalization_image"] + ) def with_integration_base_java_and_normalization(context: PipelineContext, build_platform: Platform) -> Container: @@ -746,7 +751,7 @@ def with_integration_base_java_and_normalization(context: PipelineContext, build .with_mounted_cache("/root/.cache/pip", pip_cache) .with_exec(["python", "-m", "ensurepip", "--upgrade"]) .with_exec(["pip3", "install", dbt_adapter_package]) - .with_directory("airbyte_normalization", with_normalization(context).directory("/airbyte")) + .with_directory("airbyte_normalization", with_normalization(context, build_platform).directory("/airbyte")) .with_workdir("airbyte_normalization") .with_exec(["sh", "-c", "mv * .."]) .with_workdir("/airbyte") diff --git a/tools/ci_connector_ops/ci_connector_ops/pipelines/builds/normalization.py b/tools/ci_connector_ops/ci_connector_ops/pipelines/builds/normalization.py index 85cc61cf79e9..404dd35df555 100644 --- a/tools/ci_connector_ops/ci_connector_ops/pipelines/builds/normalization.py +++ b/tools/ci_connector_ops/ci_connector_ops/pipelines/builds/normalization.py @@ -5,6 +5,7 @@ from ci_connector_ops.pipelines.actions import environments from ci_connector_ops.pipelines.bases import Step, StepResult, StepStatus from ci_connector_ops.pipelines.contexts import ConnectorContext +from dagger import Platform # TODO this class could be deleted @@ -12,7 +13,7 @@ class BuildOrPullNormalization(Step): """A step to build or pull the normalization image for a connector according to the image name.""" - def __init__(self, context: ConnectorContext, normalization_image: str) -> None: + def __init__(self, context: ConnectorContext, normalization_image: str, build_platform: Platform) -> None: """Initialize the step to build or pull the normalization image. Args: @@ -20,13 +21,14 @@ def __init__(self, context: ConnectorContext, normalization_image: str) -> None: normalization_image (str): The normalization image to build (if :dev) or pull. """ super().__init__(context) + self.build_platform = build_platform self.use_dev_normalization = normalization_image.endswith(":dev") self.normalization_image = normalization_image self.title = f"Build {self.normalization_image}" if self.use_dev_normalization else f"Pull {self.normalization_image}" async def _run(self) -> StepResult: if self.use_dev_normalization: - build_normalization_container = environments.with_normalization(self.context) + build_normalization_container = environments.with_normalization(self.context, self.build_platform) else: build_normalization_container = self.context.dagger_client.container().from_(self.normalization_image) return StepResult(self, StepStatus.SUCCESS, output_artifact=build_normalization_container) diff --git a/tools/ci_connector_ops/ci_connector_ops/pipelines/tests/java_connectors.py b/tools/ci_connector_ops/ci_connector_ops/pipelines/tests/java_connectors.py index d26bfbe02c23..003c4eb01507 100644 --- a/tools/ci_connector_ops/ci_connector_ops/pipelines/tests/java_connectors.py +++ b/tools/ci_connector_ops/ci_connector_ops/pipelines/tests/java_connectors.py @@ -83,7 +83,7 @@ async def run_all_tests(context: ConnectorContext) -> List[StepResult]: if context.connector.supports_normalization: normalization_image = f"{context.connector.normalization_repository}:dev" context.logger.info(f"This connector supports normalization: will build {normalization_image}.") - build_normalization_results = await BuildOrPullNormalization(context, normalization_image).run() + build_normalization_results = await BuildOrPullNormalization(context, normalization_image, LOCAL_BUILD_PLATFORM).run() normalization_container = build_normalization_results.output_artifact normalization_tar_file, _ = await export_container_to_tarball( context, normalization_container, tar_file_name=f"{context.connector.normalization_repository}_{context.git_revision}.tar"