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 e5ad1755fa578..6bc2fcec059d7 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,7 +11,6 @@ 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; @@ -33,14 +32,8 @@ 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(); - 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.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 334fae8684429..2bc57eedfcda8 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,7 +10,6 @@ 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; @@ -28,7 +27,6 @@ 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 { @@ -51,11 +49,6 @@ 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 a9ca83f66389d..2f2911cb51acf 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,7 +11,6 @@ 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; @@ -34,7 +33,6 @@ 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 { @@ -62,11 +60,6 @@ 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 f61cfea63b9ad..4f59898e9268b 100755 --- a/airbyte-integrations/bases/base-java/run_with_normalization.sh +++ b/airbyte-integrations/bases/base-java/run_with_normalization.sh @@ -6,34 +6,10 @@ 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' && test "$USE_1S1T_FORMAT" != "true" +elif test "$NORMALIZATION_TECHNIQUE" = 'LEGACY' 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 deleted file mode 100644 index bd27cb2255f36..0000000000000 --- a/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/DestinationConfig.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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 68bbedb3051fe..3fba41ef0f54f 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,8 +144,6 @@ 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 8d4bec36f3fa4..4b3a4896dc4a2 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,11 +20,4 @@ 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 deleted file mode 100644 index a3fff99346e10..0000000000000 --- a/airbyte-integrations/bases/base-java/src/main/java/io/airbyte/integrations/base/TypingAndDedupingFlag.java +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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 3350f38bdc3bf..efdbd2a019cdf 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,7 +7,6 @@ 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; @@ -93,13 +92,7 @@ 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, @@ -112,27 +105,7 @@ public BufferedStreamConsumer(final Consumer outputRecordCollect onClose, catalog, isValidRecord, - 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); + Duration.ofMinutes(15)); } /* @@ -146,8 +119,7 @@ public BufferedStreamConsumer(final Consumer outputRecordCollect final OnCloseFunction onClose, final ConfiguredAirbyteCatalog catalog, final CheckedFunction isValidRecord, - final Duration flushFrequency, - final String defaultNamespace) { + final Duration flushFrequency) { this.outputRecordCollector = outputRecordCollector; this.hasStarted = false; this.hasClosed = false; @@ -160,7 +132,6 @@ public BufferedStreamConsumer(final Consumer outputRecordCollect this.bufferingStrategy = bufferingStrategy; this.stateManager = new DefaultDestStateLifecycleManager(); this.bufferFlushFrequency = flushFrequency; - this.defaultNamespace = defaultNamespace; } @Override @@ -186,12 +157,7 @@ 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(); - // TODO maybe only do this in 1s1t mode? - if (Strings.isNullOrEmpty(record.getNamespace())) { - record.setNamespace(defaultNamespace); - } - final AirbyteStreamNameNamespacePair stream; - stream = AirbyteStreamNameNamespacePair.fromRecordMessage(record); + final AirbyteStreamNameNamespacePair 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 deleted file mode 100644 index 00182f989c26c..0000000000000 --- a/airbyte-integrations/bases/base-java/src/test/java/io/airbyte/integrations/base/DestinationConfigTest.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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 d6491eef6058a..11ef6402e14ec 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,8 +358,7 @@ private BufferedStreamConsumer getConsumerWithFlushFrequency() { onClose, CATALOG, isValidRecord, - Duration.ofSeconds(PERIODIC_BUFFER_FREQUENCY), - null); + Duration.ofSeconds(PERIODIC_BUFFER_FREQUENCY)); return flushFrequencyConsumer; } diff --git a/airbyte-integrations/bases/base-typing-deduping/build.gradle b/airbyte-integrations/bases/base-typing-deduping/build.gradle deleted file mode 100644 index 1381b8b458016..0000000000000 --- a/airbyte-integrations/bases/base-typing-deduping/build.gradle +++ /dev/null @@ -1,7 +0,0 @@ -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 deleted file mode 100644 index 7c6dcc28597d2..0000000000000 --- a/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/AirbyteType.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * Copyright (c) 2023 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.integrations.base.destination.typing_deduping; - -import com.fasterxml.jackson.databind.JsonNode; -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) { - 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")) { - final JsonNode items = schema.get("items"); - return new Array(fromJsonSchema(items)); - } - } else if (topLevelType.isArray()) { - final List elements = new ArrayList<>(); - topLevelType.elements().forEachRemaining(element -> { - // ignore "null" type - if (!element.asText("").equals("null")) { - elements.add(element); - } - }); - - // we encounter an array of types that actually represents a single type rather than a OneOf - if (elements.size() == 1) { - if (elements.get(0).asText("").equals("object")) { - return getStruct(schema); - } else if (elements.get(0).asText("").equals("array")) { - final JsonNode items = schema.get("items"); - return new Array(fromJsonSchema(items)); - } else { - return AirbyteTypeUtils.getAirbyteProtocolType(schema); - } - } - - final List typeOptions = elements.stream().map(AirbyteType::fromJsonSchema).toList(); - return new OneOf(typeOptions); - } - } 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); - } - - private static Struct getStruct(final JsonNode schema) { - final LinkedHashMap propertiesMap = new LinkedHashMap<>(); - final JsonNode properties = schema.get("properties"); - properties.fields().forEachRemaining(property -> { - final String key = property.getKey(); - final JsonNode value = property.getValue(); - propertiesMap.put(key, fromJsonSchema(value)); - }); - return new Struct(propertiesMap); - } - - 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) { - throw new IllegalArgumentException("Can't extract columns from a schema with multiple object options"); - } - - return (options.stream().filter(o -> o instanceof Struct).findFirst()) - .map(o -> ((Struct) o).properties()) - .orElseThrow(() -> new IllegalArgumentException("Can't extract columns from a schema with no object options")); - } - - } - -} 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 deleted file mode 100644 index b391aa0c76034..0000000000000 --- a/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/AirbyteTypeUtils.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * 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 java.util.Objects; -import java.util.stream.Stream; -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) { - final JsonNode propertyType = node.get("type"); - final JsonNode airbyteType = node.get("airbyte_type"); - final JsonNode format = node.get("format"); - if (Stream.of(propertyType, airbyteType, format).allMatch(Objects::isNull)) { - return AirbyteProtocolType.matches(node.asText()); - } - - 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 deleted file mode 100644 index 65137d1e4f4cc..0000000000000 --- a/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/CatalogParser.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * 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 deleted file mode 100644 index b6c14d31ef184..0000000000000 --- a/airbyte-integrations/bases/base-typing-deduping/src/main/java/io/airbyte/integrations/base/destination/typing_deduping/SqlGenerator.java +++ /dev/null @@ -1,126 +0,0 @@ -/* - * 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/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 deleted file mode 100644 index 5896e8b1ede2d..0000000000000 --- a/airbyte-integrations/bases/base-typing-deduping/src/test/java/io/airbyte/integrations/base/destination/typing_deduping/AirbyteTypeTest.java +++ /dev/null @@ -1,395 +0,0 @@ -/* - * 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.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 String structSchema = """ - { - "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" - } - } - } - """; - - 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); - assertEquals(struct, AirbyteType.fromJsonSchema(Jsons.deserialize(structSchema))); - } - - @Test - public void testStructSingletonListDecl() { - final String structSchema = """ - { - "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"] - } - } - } - """; - - 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); - assertEquals(struct, AirbyteType.fromJsonSchema(Jsons.deserialize(structSchema))); - } - - @Test - public void testStructNullableSingletonListDecl() { - final String structSchema = """ - { - "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); - assertEquals(struct, AirbyteType.fromJsonSchema(Jsons.deserialize(structSchema))); - } - - @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 String arraySchema = """ - { - "type": "array", - "items": { - "type": "string", - "format": "date-time", - "airbyte_type": "timestamp_with_timezone" - } - } - """; - - final AirbyteType array = new Array(AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE); - assertEquals(array, AirbyteType.fromJsonSchema(Jsons.deserialize(arraySchema))); - } - - @Test - public void testArraySingletonListDecl() { - final String arraySchema = """ - { - "type": ["array"], - "items": { - "type": ["string"], - "format": "date-time", - "airbyte_type": "timestamp_with_timezone" - } - } - """; - - final AirbyteType array = new Array(AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE); - assertEquals(array, AirbyteType.fromJsonSchema(Jsons.deserialize(arraySchema))); - } - - @Test - public void testArrayNullableSingletonListDecl() { - final String arraySchema = """ - { - "type": ["null", "array"], - "items": { - "type": ["null", "string"], - "format": "date-time", - "airbyte_type": "timestamp_with_timezone" - } - } - """; - - final AirbyteType array = new Array(AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE); - assertEquals(array, AirbyteType.fromJsonSchema(Jsons.deserialize(arraySchema))); - } - - @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 testEmpty() { - final String emptySchema = "{}"; - assertEquals(AirbyteProtocolType.UNKNOWN, AirbyteType.fromJsonSchema(Jsons.deserialize(emptySchema))); - } - - @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 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)); - } - -} 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 deleted file mode 100644 index 700f3969dfc53..0000000000000 --- a/airbyte-integrations/bases/base-typing-deduping/src/test/java/io/airbyte/integrations/base/destination/typing_deduping/CatalogParserTest.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * 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/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 0263d7c2b6107..6e2857b30ed11 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 = Strings.addRandomSuffix("sourcenamespace", "_", 8); + final var namespace1 = "sourcenamespace"; catalog.getStreams().forEach(stream -> stream.setNamespace(namespace1)); final var diffNamespaceStreams = new ArrayList(); - final var namespace2 = Strings.addRandomSuffix("diff_sourcenamespace", "_", 8);; + final var namespace2 = "diff_source_namespace"; 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/build.gradle b/airbyte-integrations/connectors/destination-bigquery/build.gradle index b785040112459..49cef6bab508c 100644 --- a/airbyte-integrations/connectors/destination-bigquery/build.gradle +++ b/airbyte-integrations/connectors/destination-bigquery/build.gradle @@ -10,10 +10,9 @@ application { } dependencies { - implementation 'com.google.cloud:google-cloud-bigquery:2.27.0' + implementation 'com.google.cloud:google-cloud-bigquery:1.122.2' 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' @@ -23,7 +22,6 @@ 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'} @@ -51,12 +49,3 @@ configurations.all { force 'com.google.api-client:google-api-client:1.31.5' } } - -integrationTestJava { - systemProperties = [ - 'junit.jupiter.execution.parallel.enabled': 'true' - // TODO what's preventing us from turning this on? (probably a lot of things) - // 'junit.jupiter.execution.parallel.mode.default': 'concurrent' - ] -} - 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 9f68f48e70fd6..7c966c3c0e054 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,6 +19,7 @@ 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 @@ -51,7 +52,7 @@ public static BufferCreateFunction createBufferFunction(final S3AvroFormatConfig return (pair, catalog) -> { final AirbyteStream stream = catalog.getStreams() .stream() - .filter(s -> s.getStream().getName().equals(pair.getName())) + .filter(s -> s.getStream().getName().equals(pair.getName()) && StringUtils.equals(s.getStream().getNamespace(), pair.getNamespace())) .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 ffba1f03cdcf0..58519e1ad4ceb 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,24 +16,17 @@ 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; @@ -73,7 +66,6 @@ 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", @@ -207,72 +199,45 @@ 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, 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); - + throws IOException { 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, parsedCatalog, outputRecordCollector, sqlGenerator); + return getStandardRecordConsumer(config, catalog, outputRecordCollector); } else { - return getGcsRecordConsumer(config, catalog, parsedCatalog, outputRecordCollector, sqlGenerator); + return getGcsRecordConsumer(config, catalog, outputRecordCollector); } } - protected Map> getUploaderMap( - final BigQuery bigquery, - final JsonNode config, - final ConfiguredAirbyteCatalog catalog, - final ParsedCatalog parsedCatalog, - final boolean use1s1t) + protected Map> getUploaderMap(final JsonNode config, + final ConfiguredAirbyteCatalog catalog) throws IOException { + final BigQuery bigquery = getBigQuery(config); + final Map> uploaderMap = new HashMap<>(); for (final ConfiguredAirbyteStream configStream : catalog.getStreams()) { final AirbyteStream stream = configStream.getStream(); - 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); + if (StringUtils.isEmpty(stream.getNamespace())) { + stream.setNamespace(BigQueryUtils.getDatasetId(config)); } - + final String streamName = stream.getName(); final UploaderConfig uploaderConfig = UploaderConfig .builder() .bigQuery(bigquery) .configStream(configStream) - .parsedStream(parsedStream) .config(config) .formatterMap(getFormatterMap(stream.getJsonSchema())) .tmpTableName(namingResolver.getTmpTableName(streamName)) - .targetTableName(targetTableName) - // This refers to whether this is BQ denormalized or not + .targetTableName(getTargetTableName(streamName)) .isDefaultAirbyteTmpSchema(isDefaultAirbyteTmpTableSchema()) .build(); @@ -313,35 +278,16 @@ protected String getTargetTableName(final String streamName) { private AirbyteMessageConsumer getStandardRecordConsumer(final JsonNode config, final ConfiguredAirbyteCatalog catalog, - final ParsedCatalog parsedCatalog, - final Consumer outputRecordCollector, - BigQuerySqlGenerator sqlGenerator) + final Consumer outputRecordCollector) throws IOException { - 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); + final Map> writeConfigs = getUploaderMap(config, catalog); + return new BigQueryRecordConsumer(writeConfigs, outputRecordCollector, BigQueryUtils.getDatasetId(config)); } public AirbyteMessageConsumer getGcsRecordConsumer(final JsonNode config, final ConfiguredAirbyteCatalog catalog, - final ParsedCatalog parsedCatalog, - final Consumer outputRecordCollector, - final BigQuerySqlGenerator sqlGenerator) - throws InterruptedException { + final Consumer outputRecordCollector) { + final StandardNameTransformer gcsNameTransformer = new GcsNameTransformer(); final BigQuery bigQuery = getBigQuery(config); final GcsDestinationConfig gcsConfig = BigQueryUtils.getGcsAvroDestinationConfig(config); @@ -377,7 +323,6 @@ 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, @@ -386,11 +331,7 @@ public AirbyteMessageConsumer getGcsRecordConsumer(final JsonNode config, onCreateBuffer, recordFormatterCreator, namingResolver::getTmpTableName, - getTargetTableNameTransformer(namingResolver), - sqlGenerator, - new BigQueryDestinationHandler(bigQuery), - parsedCatalog, - BigQueryUtils.getDatasetId(config)); + getTargetTableNameTransformer(namingResolver)); } protected BiFunction getAvroSchemaCreator() { @@ -408,7 +349,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 f86926b7d8bee..aa3069a06b042 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,31 +4,16 @@ 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.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.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -39,96 +24,24 @@ */ 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); - public static final int RECORDS_PER_TYPING_AND_DEDUPING_BATCH = 10_000; - private final BigQuery bigquery; private final Map> uploaderMap; private final Consumer outputRecordCollector; - private final String defaultDatasetId; - private final BigQuerySqlGenerator sqlGenerator; - private final BigQueryDestinationHandler destinationHandler; + private final String datasetId; private AirbyteMessage lastStateMessage = null; - // This is super hacky, but in async land we don't need to make this decision at all. We'll just run - // T+D whenever we commit raw data. - private final AtomicLong recordsSinceLastTDRun = new AtomicLong(0); - private final ParsedCatalog catalog; - private final boolean use1s1t; - private final Map overwriteStreamsWithTmpTable; - public BigQueryRecordConsumer(final BigQuery bigquery, - final Map> uploaderMap, + public BigQueryRecordConsumer(final Map> uploaderMap, final Consumer outputRecordCollector, - final String defaultDatasetId, - final BigQuerySqlGenerator sqlGenerator, - final BigQueryDestinationHandler destinationHandler, - final ParsedCatalog catalog) { - this.bigquery = bigquery; + final String datasetId) { this.uploaderMap = uploaderMap; this.outputRecordCollector = outputRecordCollector; - 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()); + this.datasetId = datasetId; } @Override - protected void startTracked() throws InterruptedException { + protected void startTracked() { // 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 (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 of the sync, wo we only do this in - // 1s1t mode. - for (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)); - } - } - } } /** @@ -141,13 +54,13 @@ protected void startTracked() throws InterruptedException { * @param message {@link AirbyteMessage} to be processed */ @Override - public void acceptTracked(final AirbyteMessage message) throws InterruptedException { + public void acceptTracked(final AirbyteMessage message) { 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(defaultDatasetId); + message.getRecord().setNamespace(datasetId); } processRecord(message); } else { @@ -161,39 +74,18 @@ public void acceptTracked(final AirbyteMessage message) throws InterruptedExcept * * @param message record to be written */ - private void processRecord(final AirbyteMessage message) throws InterruptedException { - final var streamId = AirbyteStreamNameNamespacePair.fromRecordMessage(message.getRecord()); - uploaderMap.get(streamId).upload(message); - - // This is just modular arithmetic written in a complicated way. We want to run T+D every - // RECORDS_PER_TYPING_AND_DEDUPING_BATCH records. - // TODO this counter should be per stream, not global. - if (recordsSinceLastTDRun.getAndUpdate(l -> (l + 1) % RECORDS_PER_TYPING_AND_DEDUPING_BATCH) == RECORDS_PER_TYPING_AND_DEDUPING_BATCH - 1) { - doTypingAndDeduping(catalog.getStream(streamId.getNamespace(), streamId.getName())); - } + private void processRecord(final AirbyteMessage message) { + final var pair = AirbyteStreamNameNamespacePair.fromRecordMessage(message.getRecord()); + uploaderMap.get(pair).upload(message); } @Override public void close(final boolean hasFailed) { LOGGER.info("Started closing all connections"); final List exceptionsThrown = new ArrayList<>(); - uploaderMap.forEach((streamId, uploader) -> { + uploaderMap.values().forEach(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); @@ -204,11 +96,4 @@ public void close(final boolean hasFailed) { } } - private void doTypingAndDeduping(final StreamConfig stream) throws InterruptedException { - 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 e3d4b98babf0e..3e6b960423786 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,22 +4,12 @@ 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.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; @@ -31,11 +21,8 @@ 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; @@ -59,76 +46,28 @@ public AirbyteMessageConsumer create(final JsonNode config, final BufferCreateFunction onCreateBuffer, final Function recordFormatterCreator, final Function tmpTableNameTransformer, - final Function targetTableNameTransformer, - final BigQuerySqlGenerator sqlGenerator, - final BigQueryDestinationHandler destinationHandler, - final ParsedCatalog parsedCatalog, - final String defaultNamespace) - throws InterruptedException { - boolean use1s1t = TypingAndDedupingFlag.isDestinationV2(); + final Function targetTableNameTransformer) { final Map writeConfigs = createWriteConfigs( config, catalog, - parsedCatalog, recordFormatterCreator, tmpTableNameTransformer, targetTableNameTransformer); - final var overwriteStreamsWithTmpTable = createFinalTables(use1s1t, parsedCatalog, destinationHandler, sqlGenerator); - ThrowingConsumer typeAndDedupeStreamFunction = typingAndDedupingStreamConsumer( - sqlGenerator, - destinationHandler, - parsedCatalog, - use1s1t, - overwriteStreamsWithTmpTable); - - ThrowingConsumer replaceFinalTableConsumer = - getReplaceFinalTableConsumer( - use1s1t, - sqlGenerator, - destinationHandler, - overwriteStreamsWithTmpTable, - parsedCatalog); - return new BufferedStreamConsumer( outputRecordCollector, onStartFunction(bigQueryGcsOperations, writeConfigs), new SerializedBufferingStrategy( onCreateBuffer, catalog, - flushBufferFunction(bigQueryGcsOperations, writeConfigs, catalog, typeAndDedupeStreamFunction)), - onCloseFunction(bigQueryGcsOperations, writeConfigs, replaceFinalTableConsumer), + flushBufferFunction(bigQueryGcsOperations, writeConfigs, catalog)), + onCloseFunction(bigQueryGcsOperations, writeConfigs), catalog, - json -> true, - defaultNamespace); - } - - @FunctionalInterface - public interface ThrowingConsumer { - - void accept(T t) throws E; - - } - - private ThrowingConsumer 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); - } - }; + json -> true); } private Map createWriteConfigs(final JsonNode config, final ConfiguredAirbyteCatalog catalog, - final ParsedCatalog parsedCatalog, final Function recordFormatterCreator, final Function tmpTableNameTransformer, final Function targetTableNameTransformer) { @@ -137,21 +76,16 @@ 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(), - internalTableNamespace, + BigQueryUtils.getSchema(config, configuredStream), BigQueryUtils.getDatasetLocation(config), tmpTableNameTransformer.apply(streamName), - targetTableName, + targetTableNameTransformer.apply(streamName), recordFormatter.getBigQuerySchema(), configuredStream.getDestinationSyncMode()); @@ -173,6 +107,7 @@ 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) { @@ -181,65 +116,21 @@ 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()); - // 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 + final String datasetId = writeConfig.datasetId(); + bigQueryGcsOperations.createSchemaIfNotExists(datasetId, writeConfig.datasetLocation()); // 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(rawDatasetId, writeConfig.streamName()); + bigQueryGcsOperations.createStageIfNotExists(datasetId, writeConfig.streamName()); // When OVERWRITE mode, truncate the destination's raw table prior to syncing data if (writeConfig.syncMode() == DestinationSyncMode.OVERWRITE) { - // TODO: this might need special handling during the migration - bigQueryGcsOperations.truncateTableIfExists(rawDatasetId, writeConfig.targetTableId(), writeConfig.tableSchema()); + bigQueryGcsOperations.truncateTableIfExists(datasetId, 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 @@ -251,14 +142,12 @@ private Map createFinalTables(boolean use1s1t, private FlushBufferFunction flushBufferFunction( final BigQueryStagingOperations bigQueryGcsOperations, final Map writeConfigs, - final ConfiguredAirbyteCatalog catalog, - final ThrowingConsumer typerDeduper) { + final ConfiguredAirbyteCatalog catalog) { 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: %s.\nKeys: %s\ncatalog: %s", pair, - writeConfigs.keySet(), Jsons.serialize(catalog))); + String.format("Message contained record from a stream that was not in the catalog. \ncatalog: %s", Jsons.serialize(catalog))); } final BigQueryWriteConfig writeConfig = writeConfigs.get(pair); @@ -275,7 +164,6 @@ 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); @@ -288,10 +176,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 ThrowingConsumer replaceFinalTableConsumer) { + final Map writeConfigs) { return (hasFailed) -> { /* * Previously the hasFailed value was used to commit any remaining staged files into destination, @@ -302,29 +190,9 @@ 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 ThrowingConsumer 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 57fe09be3282a..8eebe36f2763e 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,14 +203,12 @@ 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(chunkingColumn) + .setField(JavaBaseConstants.COLUMN_NAME_EMITTED_AT) .build(); final Clustering clustering = Clustering.newBuilder() - .setFields(ImmutableList.of(chunkingColumn)) + .setFields(ImmutableList.of(JavaBaseConstants.COLUMN_NAME_EMITTED_AT)) .build(); final StandardTableDefinition tableDefinition = @@ -223,7 +221,6 @@ 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); @@ -356,6 +353,15 @@ 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 a6e7c0d489545..1d80483303130 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,16 +46,6 @@ 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 2456d81248562..bd6756e94b085 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,10 +11,8 @@ 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; @@ -25,42 +23,21 @@ */ public class DefaultBigQueryRecordFormatter extends BigQueryRecordFormatter { - public static final com.google.cloud.bigquery.Schema SCHEMA = com.google.cloud.bigquery.Schema.of( + private 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) { - 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))); - } + 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) { @@ -77,11 +54,7 @@ protected Object getData(AirbyteRecordMessage recordMessage) { @Override public Schema getBigQuerySchema(JsonNode jsonSchema) { - if (TypingAndDedupingFlag.isDestinationV2()) { - return SCHEMA_V2; - } else { - return SCHEMA; - } + 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 08e284f268788..44425f33bb280 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,11 +17,6 @@ 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 deleted file mode 100644 index 287d1bd573d1a..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Copyright (c) 2023 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.integrations.destination.bigquery.typing_deduping; - -import com.google.cloud.bigquery.BigQuery; -import com.google.cloud.bigquery.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 deleted file mode 100644 index f27305c84dfc7..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.java +++ /dev/null @@ -1,464 +0,0 @@ -/* - * 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; - - // Bigquery columns are case insensitive, so do all our validation on the lowercased name - String canonicalized = name.toLowerCase(); - - // Column names aren't allowed to start with certain strings. Prepend an underscore if this happens. - if (canonicalized.startsWith("_table_") - || canonicalized.startsWith("_file_") - || canonicalized.startsWith("_partition_") - || canonicalized.startsWith("_row_timestamp_") - // yes, there are two underscores here. That's intentional. - || canonicalized.startsWith("__root__") - || canonicalized.startsWith("_colidentifier_")) { - 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()); - String dedupFinalTable = ""; - String dedupRawTable = ""; - if (stream.destinationSyncMode() == DestinationSyncMode.APPEND_DEDUP) { - dedupRawTable = dedupRawTable(stream.id(), finalSuffix, stream.columns()); - // If we're in dedup mode, then we must have a cursor - dedupFinalTable = dedupFinalTable(stream.id(), finalSuffix, stream.primaryKey(), stream.cursor().get(), 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, - "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} - - ${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) { - 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")); - - // Note that we intentionally excluded deleted records from this insert. See dedupRawRecords for an - // explanation of how CDC deletes work. - return new StringSubstitutor(Map.of( - "raw_table_id", id.rawTableId(QUOTE), - "final_table_id", id.finalTableId(finalSuffix, QUOTE), - "column_casts", columnCasts, - "column_errors", columnErrors, - "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 - AND ( - JSON_QUERY(`_airbyte_data`, '$._ab_cdc_deleted_at') IS NULL - OR JSON_TYPE(JSON_QUERY(`_airbyte_data`, '$._ab_cdc_deleted_at')) = 'null' - ) - ) - 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 LinkedHashMap streamColumns) { - final String pkList = primaryKey.stream().map(columnId -> columnId.name(QUOTE)).collect(joining(",")); - final String pkCastList = streamColumns.entrySet().stream() - .filter(e -> primaryKey.contains(e.getKey())) - .map(e -> extractAndCast(e.getKey(), e.getValue())) - .collect(joining(",\n ")); - final String cursorCast = extractAndCast(cursor, streamColumns.get(cursor)); - - // See dedupRawTable for an explanation of why we delete records using the raw data rather than the - // final table's _ab_cdc_deleted_at column. - return new StringSubstitutor(Map.of( - "raw_table_id", id.rawTableId(QUOTE), - "final_table_id", id.finalTableId(finalSuffix, QUOTE), - "pk_list", pkList, - "pk_cast_list", pkCastList, - "cursor_name", cursor.name(QUOTE), - "cursor_cast", cursorCast)).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 - ) - OR ( - ${pk_list} IN ( - SELECT ( - ${pk_cast_list} - ) - FROM ${raw_table_id} - WHERE - JSON_QUERY(`_airbyte_data`, '$._ab_cdc_deleted_at') IS NOT NULL - AND JSON_TYPE(JSON_QUERY(`_airbyte_data`, '$._ab_cdc_deleted_at')) != 'null' - AND ${cursor_name} < ${cursor_cast} - ) - );"""); - } - - @VisibleForTesting - String dedupRawTable(final StreamId id, final String finalSuffix, LinkedHashMap streamColumns) { - /* - * Note that we need to keep the deletion raw records because of how async syncs work. Consider this - * sequence of source events: 1. Insert record id=1 2. Update record id=1 3. Delete record id=1 - * - * It's possible for the destination to receive them out of order, e.g.: 1. Insert 2. Delete 3. - * Update - * - * We can generally resolve this using the cursor column (e.g. multiple updates in the wrong order). - * However, deletions are special because we propagate them as hard deletes to the final table. As a - * result, we need to keep the deletion in the raw table, so that a late-arriving update doesn't - * incorrectly reinsert the final record. - */ - String cdcDeletedAtClause; - if (streamColumns.containsKey(CDC_DELETED_AT_COLUMN)) { - cdcDeletedAtClause = """ - AND ( - JSON_QUERY(`_airbyte_data`, '$._ab_cdc_deleted_at') IS NULL - OR JSON_TYPE(JSON_QUERY(`_airbyte_data`, '$._ab_cdc_deleted_at')) = 'null' - ) - """; - } else { - cdcDeletedAtClause = ""; - } - - return new StringSubstitutor(Map.of( - "raw_table_id", id.rawTableId(QUOTE), - "final_table_id", id.finalTableId(finalSuffix, QUOTE), - "cdc_deleted_at_clause", cdcDeletedAtClause)).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} - ) - ${cdc_deleted_at_clause} - ;"""); - } - - @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 83fc51c0e46ca..f61a2a9bd023e 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,10 +14,7 @@ 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.destination.bigquery.BigQueryUtils; @@ -41,16 +38,13 @@ 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, final T writer, final WriteDisposition syncMode, final BigQuery bigQuery, - final BigQueryRecordFormatter recordFormatter, - final boolean use1s1t) { - this.use1s1t = use1s1t; + final BigQueryRecordFormatter recordFormatter) { this.table = table; this.tmpTable = tmpTable; this.writer = writer; @@ -102,20 +96,9 @@ public void close(final boolean hasFailed, final Consumer output protected void uploadData(final Consumer outputRecordCollector, final AirbyteMessage lastStateMessage) throws Exception { try { - 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()); - } else { - // Otherwise, we just need to ensure that this table exists. - // TODO alter an existing raw table? - final Table rawTable = bigQuery.getTable(table); - if (rawTable == null) { - bigQuery.create(TableInfo.newBuilder(table, StandardTableDefinition.of(recordFormatter.getBigQuerySchema())).build()); - } - } - + 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) { diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/AbstractGscBigQueryUploader.java b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/AbstractGscBigQueryUploader.java index 1ed1a963a0e01..97cbda3763ad8 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/AbstractGscBigQueryUploader.java +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/java/io/airbyte/integrations/destination/bigquery/uploader/AbstractGscBigQueryUploader.java @@ -39,8 +39,7 @@ public abstract class AbstractGscBigQueryUploader getUploader(final UploaderConfig uploaderConfig) throws IOException { - boolean use1s1t = TypingAndDedupingFlag.isDestinationV2(); - 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 schemaName = BigQueryUtils.getSchema(uploaderConfig.getConfig(), uploaderConfig.getConfigStream()); final String datasetLocation = BigQueryUtils.getDatasetLocation(uploaderConfig.getConfig()); - final Set existingDatasets = new HashSet<>(); + final Set existingSchemas = new HashSet<>(); final BigQueryRecordFormatter recordFormatter = uploaderConfig.getFormatter(); final Schema bigQuerySchema = recordFormatter.getBigQuerySchema(); - final TableId targetTable = TableId.of(dataset, uploaderConfig.getTargetTableName()); - final TableId tmpTable = TableId.of(dataset, uploaderConfig.getTmpTableName()); + final TableId targetTable = TableId.of(schemaName, uploaderConfig.getTargetTableName()); + final TableId tmpTable = TableId.of(schemaName, uploaderConfig.getTmpTableName()); BigQueryUtils.createSchemaAndTableIfNeeded( uploaderConfig.getBigQuery(), - existingDatasets, - dataset, + existingSchemas, + schemaName, tmpTable, datasetLocation, bigQuerySchema); @@ -101,8 +88,7 @@ public static AbstractBigQueryUploader getUploader(final UploaderConfig uploa uploaderConfig.getBigQuery(), syncMode, datasetLocation, - recordFormatter, - use1s1t)); + recordFormatter)); } private static AbstractGscBigQueryUploader getGcsBigQueryUploader( @@ -158,13 +144,10 @@ private static BigQueryDirectUploader getBigQueryDirectUploader( final BigQuery bigQuery, final JobInfo.WriteDisposition syncMode, final String datasetLocation, - final BigQueryRecordFormatter formatter, - final boolean use1s1t) { + final BigQueryRecordFormatter formatter) { // https://cloud.google.com/bigquery/docs/loading-data-local#loading_data_from_a_local_data_source - final TableId tableToWriteRawData = use1s1t ? targetTable : tmpTable; - LOGGER.info("Will write raw data to {} with schema {}", tableToWriteRawData, formatter.getBigQuerySchema()); final WriteChannelConfiguration writeChannelConfiguration = - WriteChannelConfiguration.newBuilder(tableToWriteRawData) + WriteChannelConfiguration.newBuilder(tmpTable) .setCreateDisposition(JobInfo.CreateDisposition.CREATE_IF_NEEDED) .setSchema(formatter.getBigQuerySchema()) .setFormatOptions(FormatOptions.json()) @@ -201,8 +184,7 @@ private static BigQueryDirectUploader getBigQueryDirectUploader( new BigQueryTableWriter(writer), syncMode, bigQuery, - formatter, - use1s1t); + formatter); } } 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 f6246989d1b4e..ab13465c9cc6b 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,7 +6,6 @@ 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; @@ -21,15 +20,7 @@ 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 9a318aca6a78e..de865f5ad80e9 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,7 +30,6 @@ 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; @@ -177,8 +176,6 @@ 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/BigQueryGcsDestinationAcceptanceTest.java b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/BigQueryGcsDestinationAcceptanceTest.java index ca29799fd3cba..5f69f0ccee362 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,7 +11,6 @@ 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; @@ -41,8 +40,6 @@ 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/BigQuerySqlGeneratorIntegrationTest.java b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGeneratorIntegrationTest.java deleted file mode 100644 index 3be1cecb08750..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGeneratorIntegrationTest.java +++ /dev/null @@ -1,1111 +0,0 @@ -/* - * 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 java.util.stream.Collectors.toSet; -import static org.junit.jupiter.api.Assertions.*; - -import com.fasterxml.jackson.databind.JsonNode; -import com.google.auth.oauth2.GoogleCredentials; -import com.google.cloud.bigquery.*; -import com.google.cloud.bigquery.Field.Mode; -import com.google.common.collect.ImmutableMap; -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.SqlGenerator.ColumnId; -import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator.StreamId; -import io.airbyte.integrations.destination.bigquery.BigQueryDestination; -import io.airbyte.integrations.destination.bigquery.BigQueryUtils; -import io.airbyte.protocol.models.v0.DestinationSyncMode; -import io.airbyte.protocol.models.v0.SyncMode; -import java.math.BigDecimal; -import java.nio.file.Files; -import java.nio.file.Path; -import java.time.Duration; -import java.time.Instant; -import java.util.*; -import java.util.Map.Entry; -import java.util.function.Function; -import java.util.stream.Collectors; -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"); - /** - * Super hacky way to sort rows represented as {@code Map} - */ - public static final Comparator> ROW_COMPARATOR = (row1, row2) -> { - int cmp; - cmp = compareRowsOnColumn(ID_COLUMN.name(), row1, row2); - if (cmp != 0) { - return cmp; - } - cmp = compareRowsOnColumn(CURSOR.name(), row1, row2); - if (cmp != 0) { - return cmp; - } - cmp = compareRowsOnColumn(CDC_CURSOR.name(), row1, row2); - return cmp; - }; - 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); - - final BigQueryOptions.Builder bigQueryBuilder = BigQueryOptions.newBuilder(); - final GoogleCredentials credentials = BigQueryDestination.getServiceAccountCredentials(config); - bq = bigQueryBuilder - .setProjectId(config.get("project_id").asText()) - .setCredentials(credentials) - .setHeaderProvider(BigQueryUtils.getHeaderProvider()) - .build() - .getService(); - } - - @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); - logAndExecute(sql); - - final TableResult result = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.finalTableId(QUOTE)).build()); - assertQueryResult( - List.of( - Map.of( - "id", Optional.of(1L), - "updated_at", Optional.of(Instant.parse("2023-01-01T01:00:00Z")), - "string", Optional.of("Alice"), - "struct", Optional.of(Jsons.deserialize( - """ - {"city": "San Francisco", "state": "CA"} - """)), - "_airbyte_extracted_at", Optional.of(Instant.parse("2023-01-01T00:00:00Z")), - "_airbyte_meta", Optional.of(Jsons.deserialize( - """ - {"errors":[]} - """))), - Map.of( - "id", Optional.of(1L), - "updated_at", Optional.of(Instant.parse("2023-01-01T02:00:00Z")), - "string", Optional.of("Alice"), - "struct", Optional.of(Jsons.deserialize( - """ - {"city": "San Diego", "state": "CA"} - """)), - "_airbyte_extracted_at", Optional.of(Instant.parse("2023-01-01T00:00:00Z")), - "_airbyte_meta", Optional.of(Jsons.deserialize( - """ - {"errors":[]} - """))), - Map.of( - "id", Optional.of(2L), - "updated_at", Optional.of(Instant.parse("2023-01-01T03:00:00Z")), - "string", Optional.of("Bob"), - "struct", Optional.empty(), - "integer", Optional.empty(), - "_airbyte_extracted_at", Optional.of(Instant.parse("2023-01-01T00:00:00Z")), - "_airbyte_meta", Optional.of(Jsons.deserialize( - """ - {"errors":["Problem with `integer`"]} - """)))), - 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, COLUMNS); - logAndExecute(sql); - - final TableResult result = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.finalTableId(QUOTE)).build()); - assertQueryResult( - List.of( - Map.of( - "id", Optional.of(1L), - "updated_at", Optional.of(Instant.parse("2023-01-01T02:00:00Z")), - "string", Optional.of("Alice"), - "struct", Optional.of(Jsons.deserialize( - """ - {"city": "San Diego", "state": "CA"} - """)), - "integer", Optional.of(84L), - "_airbyte_extracted_at", Optional.of(Instant.parse("2023-01-01T00:00:00Z")), - "_airbyte_meta", Optional.of(Jsons.deserialize( - """ - {"errors":[]} - """))), - Map.of( - "id", Optional.of(2L), - "updated_at", Optional.of(Instant.parse("2023-01-01T03:00:00Z")), - "string", Optional.of("Bob"), - "struct", Optional.empty(), - "_airbyte_extracted_at", Optional.of(Instant.parse("2023-01-01T00:00:00Z")), - "_airbyte_meta", Optional.of(Jsons.deserialize( - """ - {"errors":["blah blah integer"]} - """)))), - 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, "", CDC_COLUMNS); - logAndExecute(sql); - - final TableResult result = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.rawTableId(QUOTE)).build()); - assertQueryResult( - List.of( - Map.of( - "_airbyte_raw_id", Optional.of("80c99b54-54b4-43bd-b51b-1f67dafa2c52"), - "_airbyte_extracted_at", Optional.of(Instant.parse("2023-01-01T00:00:00Z")), - "_airbyte_data", Optional.of(Jsons.deserialize( - """ - {"id": 1, "updated_at": "2023-01-01T02:00:00Z", "string": "Alice", "struct": {"city": "San Diego", "state": "CA"}, "integer": 84} - """))), - Map.of( - "_airbyte_raw_id", Optional.of("ad690bfb-c2c2-4172-bd73-a16c86ccbb67"), - "_airbyte_extracted_at", Optional.of(Instant.parse("2023-01-01T00:00:00Z")), - "_airbyte_data", Optional.of(Jsons.deserialize( - """ - {"id": 2, "updated_at": "2023-01-01T03:00:00Z", "string": "Bob", "integer": "oops"} - """)))), - 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()); - assertQueryResult( - List.of( - new ImmutableMap.Builder>() - .put("id", Optional.of(1L)) - .put("updated_at", Optional.of(Instant.parse("2023-01-01T01:00:00Z"))) - .put("array", Optional.of(Jsons.deserialize( - """ - ["foo"] - """))) - .put("struct", Optional.of(Jsons.deserialize( - """ - {"foo": "bar"} - """))) - .put("string", Optional.of("foo")) - .put("number", Optional.of(new BigDecimal("42.1"))) - .put("integer", Optional.of(42L)) - .put("boolean", Optional.of(true)) - .put("timestamp_with_timezone", Optional.of(Instant.parse("2023-01-23T12:34:56Z"))) - .put("timestamp_without_timezone", Optional.of("2023-01-23T12:34:56")) - .put("time_with_timezone", Optional.of("12:34:56Z")) - .put("time_without_timezone", Optional.of("12:34:56")) - .put("date", Optional.of("2023-01-23")) - .put("_airbyte_extracted_at", Optional.of(Instant.parse("2023-01-01T00:00:00Z"))) - .put("_airbyte_meta", Optional.of(Jsons.deserialize( - """ - {"errors":[]} - """))) - .build(), - new ImmutableMap.Builder>() - .put("id", Optional.of(2L)) - .put("updated_at", Optional.of(Instant.parse("2023-01-01T01:00:00Z"))) - .put("array", Optional.empty()) - .put("struct", Optional.empty()) - .put("string", Optional.empty()) - .put("number", Optional.empty()) - .put("integer", Optional.empty()) - .put("boolean", Optional.empty()) - .put("timestamp_with_timezone", Optional.empty()) - .put("timestamp_without_timezone", Optional.empty()) - .put("time_with_timezone", Optional.empty()) - .put("time_without_timezone", Optional.empty()) - .put("date", Optional.empty()) - .put("_airbyte_extracted_at", Optional.of(Instant.parse("2023-01-01T00:00:00Z"))) - .put("_airbyte_meta", Optional.of(Jsons.deserialize( - """ - {"errors":[]} - """))) - .build(), - new ImmutableMap.Builder>() - .put("id", Optional.of(3L)) - .put("updated_at", Optional.of(Instant.parse("2023-01-01T01:00:00Z"))) - .put("array", Optional.empty()) - .put("struct", Optional.empty()) - .put("string", Optional.empty()) - .put("number", Optional.empty()) - .put("integer", Optional.empty()) - .put("boolean", Optional.empty()) - .put("timestamp_with_timezone", Optional.empty()) - .put("timestamp_without_timezone", Optional.empty()) - .put("time_with_timezone", Optional.empty()) - .put("time_without_timezone", Optional.empty()) - .put("date", Optional.empty()) - .put("_airbyte_extracted_at", Optional.of(Instant.parse("2023-01-01T00:00:00Z"))) - .put("_airbyte_meta", Optional.of(Jsons.deserialize( - """ - {"errors":[]} - """))) - .build(), - new ImmutableMap.Builder>() - .put("id", Optional.of(4L)) - .put("updated_at", Optional.of(Instant.parse("2023-01-01T01:00:00Z"))) - .put("array", Optional.empty()) - .put("struct", Optional.empty()) - .put("string", Optional.empty()) - .put("number", Optional.empty()) - .put("integer", Optional.empty()) - .put("boolean", Optional.empty()) - .put("timestamp_with_timezone", Optional.empty()) - .put("timestamp_without_timezone", Optional.empty()) - .put("time_with_timezone", Optional.empty()) - .put("time_without_timezone", Optional.empty()) - .put("date", Optional.empty()) - .put("_airbyte_extracted_at", Optional.of(Instant.parse("2023-01-01T00:00:00Z"))) - .put("_airbyte_meta", Optional.of(Jsons.deserialize( - """ - {"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`" - ]} - """))) - .build()), - 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 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 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); - - // TODO - final long finalRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.finalTableId("", QUOTE)).build()).getTotalRows(); - assertEquals(4, finalRows); - final long rawRows = bq.query(QueryJobConfiguration.newBuilder("SELECT * FROM " + streamId.rawTableId(QUOTE)).build()).getTotalRows(); - // Explanation: - // id=0 has two raw records (the old deletion record + zombie_returned) - // id=1 has one raw record (the new deletion record; the old raw record was deleted) - // id=2 has one raw record (the newer alice2 record) - // id=3 has one raw record - // id=4 has one raw record - // id=5 has one raw deletion record - assertEquals(7, 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. 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); - - // 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); - } - - /** - * 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(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); - } - - 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()); - } - - private Map toMap(Schema schema, FieldValueList row) { - final Map map = new HashMap<>(); - for (int i = 0; i < schema.getFields().size(); i++) { - final Field field = schema.getFields().get(i); - final FieldValue value = row.get(i); - Object typedValue; - if (value.getValue() == null) { - typedValue = null; - } else { - typedValue = switch (field.getType().getStandardType()) { - case BOOL -> value.getBooleanValue(); - case INT64 -> value.getLongValue(); - case FLOAT64 -> value.getDoubleValue(); - case NUMERIC, BIGNUMERIC -> value.getNumericValue(); - case STRING -> value.getStringValue(); - case BYTES -> value.getBytesValue(); - case TIMESTAMP -> value.getTimestampInstant(); - // value.getTimestampInstant() fails to parse these types - case DATE, DATETIME, TIME -> value.getStringValue(); - // bigquery returns JSON columns as string; manually parse it into a JsonNode - case JSON -> Jsons.deserialize(value.getStringValue()); - - // Default case for weird types (struct, array, geography, interval) - default -> value.getStringValue(); - }; - } - map.put(field.getName(), typedValue); - } - return map; - } - - /** - * Asserts that the expected rows match the query result. Please don't read this code. Trust the - * logs. - */ - private void assertQueryResult(final List>> expectedRows, final TableResult result) { - List> actualRows = result.streamAll().map(row -> toMap(result.getSchema(), row)).toList(); - List>> missingRows = new ArrayList<>(); - Set> matchedRows = new HashSet<>(); - boolean foundMultiMatch = false; - // For each expected row, iterate through all actual rows to find a match. - for (Map> expectedRow : expectedRows) { - final List> matchingRows = actualRows.stream().filter(actualRow -> { - // We only want to check the fields that are specified in the expected row. - // E.g.we shouldn't assert against randomized UUIDs. - for (Entry> expectedEntry : expectedRow.entrySet()) { - // If the expected value is empty, we just check that the actual value is null. - if (expectedEntry.getValue().isEmpty()) { - if (actualRow.get(expectedEntry.getKey()) != null) { - // It wasn't null, so this actualRow doesn't match the expected row - return false; - } else { - // It _was_ null, so we can move on the next key. - continue; - } - } - // If the expected value is non-empty, we check that the actual value matches. - if (!expectedEntry.getValue().get().equals(actualRow.get(expectedEntry.getKey()))) { - return false; - } - } - return true; - }).toList(); - - if (matchingRows.size() == 0) { - missingRows.add(expectedRow); - } else if (matchingRows.size() > 1) { - foundMultiMatch = true; - } - matchedRows.addAll(matchingRows); - } - - // TODO is the foundMultiMatch condition correct? E.g. what if we try to write the same row twice - // (because of a retry)? Are we - // guaranteed to have some differentiator? - if (foundMultiMatch || !missingRows.isEmpty() || matchedRows.size() != actualRows.size()) { - Set> extraRows = actualRows.stream().filter(row -> !matchedRows.contains(row)).collect(toSet()); - fail(diff(missingRows, extraRows)); - } - } - - private static String sortedToString(Map record) { - return sortedToString(record, Function.identity()); - } - - private static String sortedToString(Map record, Function valueMapper) { - return "{" - + record.entrySet().stream() - .sorted(Entry.comparingByKey()) - .map(entry -> entry.getKey() + "=" + valueMapper.apply(entry.getValue())) - .collect(Collectors.joining(", ")) - + "}"; - } - - /** - * Attempts to generate a pretty-print diff of the rows. Output will look something like: - * {@code Missing row: {id=1} Extra row: {id=2} Mismatched row: id=3; foo_column expected String - * arst, got Long 42 } - * - * Assumes that rows with the same id and cursor are the same row. - */ - private static String diff(List>> missingRowsRaw, Set> extraRowsRaw) { - List> missingRows = missingRowsRaw.stream() - .map(row -> { - // Extract everything from inside the optionals. - Map newRow = new HashMap<>(); - for (Entry> entry : row.entrySet()) { - newRow.put(entry.getKey(), entry.getValue().orElse(null)); - } - return newRow; - }).sorted(ROW_COMPARATOR) - .toList(); - - List> extraRows = extraRowsRaw.stream().sorted(ROW_COMPARATOR).toList(); - - String output = ""; - int missingIndex = 0; - int extraIndex = 0; - while (missingIndex < missingRows.size() && extraIndex < extraRows.size()) { - Map missingRow = missingRows.get(missingIndex); - Map extraRow = extraRows.get(extraIndex); - int compare = ROW_COMPARATOR.compare(missingRow, extraRow); - if (compare < 0) { - // missing row is too low - we should print missing rows until we catch up - output += "Missing row: " + sortedToString(missingRow) + "\n"; - missingIndex++; - } else if (compare == 0) { - // rows match - we should print the diff between them - output += "Mismatched row: "; - if (missingRow.containsKey(ID_COLUMN.name())) { - output += "id=" + missingRow.get(ID_COLUMN.name()) + "; "; - } - if (missingRow.containsKey(CURSOR.name())) { - output += "updated_at=" + missingRow.get(CURSOR.name()) + "; "; - } - if (missingRow.containsKey(CDC_CURSOR.name())) { - output += "_ab_cdc_lsn=" + missingRow.get(CDC_CURSOR.name()) + "; "; - } - output += "\n"; - for (String key : missingRow.keySet().stream().sorted().toList()) { - Object missingValue = missingRow.get(key); - Object extraValue = extraRow.get(key); - if (!Objects.equals(missingValue, extraValue)) { - output += " " + key + " expected " + getClassAndValue(missingValue) + ", got " + getClassAndValue(extraValue) + "\n"; - } - } - - missingIndex++; - extraIndex++; - } else { - // extra row is too low - we should print extra rows until we catch up - output += "Extra row: " + sortedToString(extraRow) + "\n"; - extraIndex++; - } - } - while (missingIndex < missingRows.size()) { - Map missingRow = missingRows.get(missingIndex); - output += "Missing row: " + sortedToString(missingRow) + "\n"; - missingIndex++; - } - while (extraIndex < extraRows.size()) { - Map extraRow = extraRows.get(extraIndex); - output += "Extra row: " + sortedToString(extraRow) + "\n"; - extraIndex++; - } - return output; - } - - /** - * Compare two rows on the given column. Sorts nulls first. If the values are not the same type, - * assumes the left value is smaller. - */ - private static int compareRowsOnColumn(String column, Map row1, Map row2) { - Comparable r1id = (Comparable) row1.get(column); - Comparable r2id = (Comparable) row2.get(column); - if (r1id == null) { - if (r2id == null) { - return 0; - } else { - return -1; - } - } else { - if (r2id == null) { - return 1; - } else { - if (r1id.getClass().equals(r2id.getClass())) { - // We're doing some very sketchy type-casting nonsense here, but it's guarded by the class equality - // check. - return ((Comparable) r1id).compareTo(r2id); - } else { - // Both values are non-null, but they're not the same type. Assume left is smaller. - return -1; - } - } - } - } - - private static String getClassAndValue(Object o) { - if (o == null) { - return null; - } else { - return o.getClass().getSimpleName() + " " + o; - } - } - -} 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 e67ee15225b45..27511e9934530 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,25 +4,15 @@ 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; @@ -34,22 +24,9 @@ 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 deleted file mode 100644 index 60063dd96de82..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/test/java/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGeneratorTest.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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 java.util.ArrayList; -import java.util.LinkedHashMap; -import org.junit.jupiter.api.Test; - -public class BigQuerySqlGeneratorTest { - - @Test - public void testToDialectType() { - final BigQuerySqlGenerator generator = new BigQuerySqlGenerator(); - 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)); - } - -} diff --git a/airbyte-integrations/connectors/destination-gcs/Dockerfile b/airbyte-integrations/connectors/destination-gcs/Dockerfile index 138775e94e0cd..847a2da7ef139 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.1 +LABEL io.airbyte.version=0.4.0 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 fe7fa51f3eb97..3aa35bf0f4e1e 100644 --- a/airbyte-integrations/connectors/destination-gcs/build.gradle +++ b/airbyte-integrations/connectors/destination-gcs/build.gradle @@ -45,12 +45,3 @@ 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 f9be343a7edd5..252cd3358bf1b 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.1 + dockerImageTag: 0.4.0 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 9ea223b318f6a..c8d22cb0ed4b8 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,7 +5,6 @@ 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; @@ -19,7 +18,6 @@ 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, @@ -32,25 +30,14 @@ 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 (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(); + + 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(); 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 fe1b3b30eb13f..52f5d68907e72 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,8 +11,6 @@ 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; @@ -28,8 +26,6 @@ class GcsAvroWriterTest { @Test public void generatesCorrectObjectPath() throws IOException { - DestinationConfig.initialize(Jsons.deserialize("{}")); - final GcsAvroWriter writer = new GcsAvroWriter( new GcsDestinationConfig( "fake-bucket", diff --git a/docs/connector-development/debugging-docker.md b/docs/connector-development/debugging-docker.md index 3f707fc0d3d88..bacedbcc0aeb6 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:5005" VERSION="dev" docker compose -f docker-compose.yaml -f docker-compose.debug.yaml up +DEBUG_CONTAINER_IMAGE="destination-postgres" 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/gcs.md b/docs/integrations/destinations/gcs.md index 5f730b613a9f6..caff5c6e2e1db 100644 --- a/docs/integrations/destinations/gcs.md +++ b/docs/integrations/destinations/gcs.md @@ -237,7 +237,6 @@ Under the hood, an Airbyte data stream in Json schema is first converted to an A | Version | Date | Pull Request | Subject | | :------ | :--------- | :--------------------------------------------------------- | :------------------------------------------------------------------------------------------------------------------------- | -| 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 | | 0.3.0 | 2023-04-28 | [#25570](https://github.com/airbytehq/airbyte/pull/25570) | Fix: all integer schemas should be converted to Avro longs | | 0.2.17 | 2023-04-27 | [#25346](https://github.com/airbytehq/airbyte/pull/25346) | Internal code cleanup | diff --git a/docs/understanding-airbyte/supported-data-types.md b/docs/understanding-airbyte/supported-data-types.md index cb11bf5de539b..ca84b13759fdc 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": "timestamp_with_timezone" + "airbyte_type": "timestampt_with_timezone" } } } diff --git a/settings.gradle b/settings.gradle index fe9e61d097f4b..54bee84c8a991 100644 --- a/settings.gradle +++ b/settings.gradle @@ -104,7 +104,6 @@ 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: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'