Revert "Generate seed connector specs on build (#7501)"
This reverts commit a534bb2a8f.
This commit is contained in:
@@ -7,9 +7,7 @@ package io.airbyte.config.init;
|
||||
public enum SeedType {
|
||||
|
||||
STANDARD_SOURCE_DEFINITION("/seed/source_definitions.yaml", "sourceDefinitionId"),
|
||||
STANDARD_DESTINATION_DEFINITION("/seed/destination_definitions.yaml", "destinationDefinitionId"),
|
||||
SOURCE_SPEC("/seed/source_specs.yaml", "dockerImage"),
|
||||
DESTINATION_SPEC("/seed/destination_specs.yaml", "dockerImage");
|
||||
STANDARD_DESTINATION_DEFINITION("/seed/destination_definitions.yaml", "destinationDefinitionId");
|
||||
|
||||
final String resourcePath;
|
||||
// ID field name
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
@@ -7,11 +7,10 @@ plugins {
|
||||
dependencies {
|
||||
implementation project(':airbyte-json-validation')
|
||||
implementation project(':airbyte-protocol:models')
|
||||
implementation project(':airbyte-commons')
|
||||
}
|
||||
|
||||
jsonSchema2Pojo {
|
||||
sourceType = SourceType.YAMLSCHEMA
|
||||
sourceType = SourceType.YAMLSCHEMA
|
||||
source = files("${sourceSets.main.output.resourcesDir}/types")
|
||||
targetDirectory = new File(project.buildDir, 'generated/src/gen/java/')
|
||||
|
||||
|
||||
@@ -1,16 +0,0 @@
|
||||
---
|
||||
"$schema": http://json-schema.org/draft-07/schema#
|
||||
"$id": https://github.com/airbytehq/airbyte/blob/master/airbyte-config/models/src/main/resources/types/DockerImageSpec.yaml
|
||||
title: DockerImageSpec
|
||||
description: docker image name and the connector specification associated with it
|
||||
type: object
|
||||
required:
|
||||
- dockerImage
|
||||
- spec
|
||||
additionalProperties: false
|
||||
properties:
|
||||
dockerImage:
|
||||
type: string
|
||||
spec:
|
||||
type: object
|
||||
existingJavaType: io.airbyte.protocol.models.ConnectorSpecification
|
||||
@@ -366,14 +366,7 @@ public class DatabaseConfigPersistence implements ConfigPersistence {
|
||||
|
||||
final ConnectorInfo connectorInfo = connectorRepositoryToIdVersionMap.get(repository);
|
||||
final JsonNode currentDefinition = connectorInfo.definition;
|
||||
|
||||
// todo (lmossman) - this logic to remove the "spec" field is temporary; it is necessary to avoid
|
||||
// breaking users who are actively using an old connector version, otherwise specs from the most
|
||||
// recent connector versions may be inserted into the db which could be incompatible with the
|
||||
// version they are actually using.
|
||||
// Once the faux major version bump has been merged, this "new field" logic will be removed
|
||||
// entirely.
|
||||
final Set<String> newFields = Sets.difference(getNewFields(currentDefinition, latestDefinition), Set.of("spec"));
|
||||
final Set<String> newFields = getNewFields(currentDefinition, latestDefinition);
|
||||
|
||||
// Process connector in use
|
||||
if (connectorRepositoriesInUse.contains(repository)) {
|
||||
|
||||
@@ -5,10 +5,8 @@
|
||||
package io.airbyte.config.persistence;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.io.Resources;
|
||||
import io.airbyte.commons.docker.DockerUtils;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.util.MoreIterators;
|
||||
import io.airbyte.commons.yaml.Yamls;
|
||||
@@ -21,7 +19,6 @@ import java.net.URL;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@@ -48,40 +45,11 @@ public class YamlSeedConfigPersistence implements ConfigPersistence {
|
||||
return new YamlSeedConfigPersistence(seedDefinitionsResourceClass);
|
||||
}
|
||||
|
||||
private YamlSeedConfigPersistence(final Class<?> seedResourceClass) throws IOException {
|
||||
final Map<String, JsonNode> sourceDefinitionConfigs = getConfigs(seedResourceClass, SeedType.STANDARD_SOURCE_DEFINITION);
|
||||
final Map<String, JsonNode> sourceSpecConfigs = getConfigs(seedResourceClass, SeedType.SOURCE_SPEC);
|
||||
final Map<String, JsonNode> fullSourceDefinitionConfigs = sourceDefinitionConfigs.entrySet().stream()
|
||||
.collect(Collectors.toMap(Entry::getKey, e -> mergeSpecIntoDefinition(e.getValue(), sourceSpecConfigs)));
|
||||
|
||||
final Map<String, JsonNode> destinationDefinitionConfigs = getConfigs(seedResourceClass, SeedType.STANDARD_DESTINATION_DEFINITION);
|
||||
final Map<String, JsonNode> destinationSpecConfigs = getConfigs(seedResourceClass, SeedType.DESTINATION_SPEC);
|
||||
final Map<String, JsonNode> fullDestinationDefinitionConfigs = destinationDefinitionConfigs.entrySet().stream()
|
||||
.collect(Collectors.toMap(Entry::getKey, e -> mergeSpecIntoDefinition(e.getValue(), destinationSpecConfigs)));
|
||||
|
||||
private YamlSeedConfigPersistence(final Class<?> seedDefinitionsResourceClass) throws IOException {
|
||||
this.allSeedConfigs = ImmutableMap.<SeedType, Map<String, JsonNode>>builder()
|
||||
.put(SeedType.STANDARD_SOURCE_DEFINITION, fullSourceDefinitionConfigs)
|
||||
.put(SeedType.STANDARD_DESTINATION_DEFINITION, fullDestinationDefinitionConfigs).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Merges the corresponding spec JSON into the definition JSON. This is necessary because specs are
|
||||
* stored in a separate resource file from definitions.
|
||||
*
|
||||
* @param definitionJson JSON of connector definition that is missing a spec
|
||||
* @param specConfigs map of docker image to JSON of docker image/connector spec pair
|
||||
* @return JSON of connector definition including the connector spec
|
||||
*/
|
||||
private JsonNode mergeSpecIntoDefinition(final JsonNode definitionJson, final Map<String, JsonNode> specConfigs) {
|
||||
final String dockerImage = DockerUtils.getTaggedImageName(
|
||||
definitionJson.get("dockerRepository").asText(),
|
||||
definitionJson.get("dockerImageTag").asText());
|
||||
final JsonNode specConfigJson = specConfigs.get(dockerImage);
|
||||
if (specConfigJson == null || specConfigJson.get("spec") == null) {
|
||||
throw new UnsupportedOperationException(String.format("There is no seed spec for docker image %s", dockerImage));
|
||||
}
|
||||
((ObjectNode) definitionJson).set("spec", specConfigJson.get("spec"));
|
||||
return definitionJson;
|
||||
.put(SeedType.STANDARD_SOURCE_DEFINITION, getConfigs(seedDefinitionsResourceClass, SeedType.STANDARD_SOURCE_DEFINITION))
|
||||
.put(SeedType.STANDARD_DESTINATION_DEFINITION, getConfigs(seedDefinitionsResourceClass, SeedType.STANDARD_DESTINATION_DEFINITION))
|
||||
.build();
|
||||
}
|
||||
|
||||
@SuppressWarnings("UnstableApiUsage")
|
||||
|
||||
@@ -15,7 +15,6 @@ import io.airbyte.config.StandardSourceDefinition;
|
||||
import io.airbyte.config.StandardSync;
|
||||
import io.airbyte.config.StandardWorkspace;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Stream;
|
||||
@@ -42,7 +41,6 @@ public class YamlSeedConfigPersistenceTest {
|
||||
assertEquals("airbyte/source-mysql", mysqlSource.getDockerRepository());
|
||||
assertEquals("https://docs.airbyte.io/integrations/sources/mysql", mysqlSource.getDocumentationUrl());
|
||||
assertEquals("mysql.svg", mysqlSource.getIcon());
|
||||
assertEquals(URI.create("https://docs.airbyte.io/integrations/sources/mysql"), mysqlSource.getSpec().getDocumentationUrl());
|
||||
|
||||
// destination
|
||||
final String s3DestinationId = "4816b78f-1489-44c1-9060-4b19d5fa9362";
|
||||
@@ -52,16 +50,13 @@ public class YamlSeedConfigPersistenceTest {
|
||||
assertEquals("S3", s3Destination.getName());
|
||||
assertEquals("airbyte/destination-s3", s3Destination.getDockerRepository());
|
||||
assertEquals("https://docs.airbyte.io/integrations/destinations/s3", s3Destination.getDocumentationUrl());
|
||||
assertEquals(URI.create("https://docs.airbyte.io/integrations/destinations/s3"), s3Destination.getSpec().getDocumentationUrl());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetInvalidConfig() {
|
||||
assertThrows(
|
||||
UnsupportedOperationException.class,
|
||||
assertThrows(UnsupportedOperationException.class,
|
||||
() -> PERSISTENCE.getConfig(ConfigSchema.STANDARD_SYNC, "invalid_id", StandardSync.class));
|
||||
assertThrows(
|
||||
ConfigNotFoundException.class,
|
||||
assertThrows(ConfigNotFoundException.class,
|
||||
() -> PERSISTENCE.getConfig(ConfigSchema.STANDARD_SOURCE_DEFINITION, "invalid_id", StandardWorkspace.class));
|
||||
}
|
||||
|
||||
|
||||
@@ -1,16 +0,0 @@
|
||||
# Generating Seed Connector Specs
|
||||
|
||||
The catalog of seeded connector definitions is stored and manually updated in the `airbyte-config/init/src/main/resources/seed/*_definitions.yaml`
|
||||
files. These manually-maintained connector definitions intentionally _do not_ contain the connector specs, in an effort to keep these files
|
||||
human-readable and easily-editable, and because specs can be automatically fetched.
|
||||
|
||||
This automatic fetching of connector specs is the goal of the SeedConnectorSpecGenerator. This class reads the connector definitions in
|
||||
the `airbyte-config/init/src/main/resources/seed/*_definitions.yaml` files, fetches the corresponding specs from the GCS bucket cache, and writes the
|
||||
specs to the `airbyte-config/init/src/main/resources/seed/*_specs.yaml` files. See the
|
||||
[SeedConnectorSpecGenerator](src/main/java/io/airbyte/config/specs/SeedConnectorSpecGenerator.java) class for more details.
|
||||
|
||||
Therefore, whenever a connector definition is updated in the `airbyte-config/init/src/main/resources/seed/*_definitions.yaml` files, the
|
||||
SeedConnectorSpecGenerator should be re-ran to generate the updated connector specs files. To do so,
|
||||
run `./gradlew :airbyte-config:init:processResources`, or just build the platform project, and commit the changes to your PR. If you do not do this,
|
||||
the build in the CI will fail because there will be a diff in the generated files as you have not checked in the changes that were applied by the
|
||||
generator.
|
||||
@@ -1,24 +0,0 @@
|
||||
plugins {
|
||||
id 'java'
|
||||
}
|
||||
|
||||
dependencies {
|
||||
implementation 'commons-cli:commons-cli:1.4'
|
||||
|
||||
implementation project(':airbyte-commons')
|
||||
implementation project(':airbyte-commons-cli')
|
||||
implementation project(':airbyte-config:models')
|
||||
implementation project(':airbyte-protocol:models')
|
||||
implementation project(':airbyte-json-validation')
|
||||
}
|
||||
|
||||
task generateSeedConnectorSpecs(type: JavaExec, dependsOn: compileJava) {
|
||||
classpath = sourceSets.main.runtimeClasspath
|
||||
|
||||
mainClass = 'io.airbyte.config.specs.SeedConnectorSpecGenerator'
|
||||
|
||||
args '--seed-root'
|
||||
args new File(project(":airbyte-config:init").projectDir, '/src/main/resources/seed')
|
||||
}
|
||||
|
||||
project(":airbyte-config:init").tasks.processResources.dependsOn(generateSeedConnectorSpecs)
|
||||
@@ -1,70 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2021 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.config.specs;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.api.client.util.Preconditions;
|
||||
import com.google.cloud.storage.Blob;
|
||||
import com.google.cloud.storage.Storage;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.protocol.models.AirbyteProtocolSchema;
|
||||
import io.airbyte.protocol.models.ConnectorSpecification;
|
||||
import io.airbyte.validation.json.JsonSchemaValidator;
|
||||
import io.airbyte.validation.json.JsonValidationException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Optional;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class GcsBucketSpecFetcher {
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(GcsBucketSpecFetcher.class);
|
||||
|
||||
private final Storage storage;
|
||||
private final String bucketName;
|
||||
|
||||
public GcsBucketSpecFetcher(final Storage storage, final String bucketName) {
|
||||
this.storage = storage;
|
||||
this.bucketName = bucketName;
|
||||
}
|
||||
|
||||
public String getBucketName() {
|
||||
return bucketName;
|
||||
}
|
||||
|
||||
public Optional<ConnectorSpecification> attemptFetch(final String dockerImage) {
|
||||
final String[] dockerImageComponents = dockerImage.split(":");
|
||||
Preconditions.checkArgument(dockerImageComponents.length == 2, "Invalidate docker image: " + dockerImage);
|
||||
final String dockerImageName = dockerImageComponents[0];
|
||||
final String dockerImageTag = dockerImageComponents[1];
|
||||
|
||||
final Path specPath = Path.of("specs").resolve(dockerImageName).resolve(dockerImageTag).resolve("spec.json");
|
||||
LOGGER.debug("Checking path for cached spec: {} {}", bucketName, specPath);
|
||||
final Blob specAsBlob = storage.get(bucketName, specPath.toString());
|
||||
|
||||
// if null it means the object was not found.
|
||||
if (specAsBlob == null) {
|
||||
LOGGER.debug("Spec not found in bucket storage");
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
final String specAsString = new String(specAsBlob.getContent(), StandardCharsets.UTF_8);
|
||||
try {
|
||||
validateConfig(Jsons.deserialize(specAsString));
|
||||
} catch (final JsonValidationException e) {
|
||||
LOGGER.error("Received invalid spec from bucket store. {}", e.toString());
|
||||
return Optional.empty();
|
||||
}
|
||||
return Optional.of(Jsons.deserialize(specAsString, ConnectorSpecification.class));
|
||||
}
|
||||
|
||||
private static void validateConfig(final JsonNode json) throws JsonValidationException {
|
||||
final JsonSchemaValidator jsonSchemaValidator = new JsonSchemaValidator();
|
||||
final JsonNode specJsonSchema = JsonSchemaValidator.getSchema(AirbyteProtocolSchema.PROTOCOL.getFile(), "ConnectorSpecification");
|
||||
jsonSchemaValidator.ensure(specJsonSchema, json);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -1,127 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2021 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.config.specs;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.cloud.storage.StorageOptions;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import io.airbyte.commons.cli.Clis;
|
||||
import io.airbyte.commons.io.IOs;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.util.MoreIterators;
|
||||
import io.airbyte.commons.yaml.Yamls;
|
||||
import io.airbyte.config.DockerImageSpec;
|
||||
import io.airbyte.config.EnvConfigs;
|
||||
import io.airbyte.protocol.models.ConnectorSpecification;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Path;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.cli.CommandLine;
|
||||
import org.apache.commons.cli.Option;
|
||||
import org.apache.commons.cli.Options;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* This script is responsible for ensuring that up-to-date {@link ConnectorSpecification}s for every
|
||||
* connector definition in the seed are stored in a corresponding resource file, for the purpose of
|
||||
* seeding the specs into the config database on server startup. See
|
||||
* ./airbyte-config/specs/readme.md for more details on how this class is run and how it fits into
|
||||
* the project.
|
||||
* <p>
|
||||
* Specs are stored in a separate file from the definitions in an effort to keep the definitions
|
||||
* yaml files human-readable and easily-editable, as specs can be rather large.
|
||||
* <p>
|
||||
* Specs are fetched from the GCS spec cache bucket, so if any specs are missing from the bucket
|
||||
* then this will fail. Note that this script only pulls specs from the bucket cache; it never
|
||||
* pushes specs to the bucket. Since this script runs at build time, the decision was to depend on
|
||||
* the bucket cache rather than running a docker container to fetch the spec during the build which
|
||||
* could be slow and unwieldy. If there is a failure, check the bucket cache and figure out how to
|
||||
* get the correct spec in there.
|
||||
*/
|
||||
public class SeedConnectorSpecGenerator {
|
||||
|
||||
private static final String DOCKER_REPOSITORY_FIELD = "dockerRepository";
|
||||
private static final String DOCKER_IMAGE_TAG_FIELD = "dockerImageTag";
|
||||
private static final String DOCKER_IMAGE_FIELD = "dockerImage";
|
||||
private static final String SPEC_FIELD = "spec";
|
||||
private static final String SPEC_BUCKET_NAME = new EnvConfigs().getSpecCacheBucket();
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(SeedConnectorSpecGenerator.class);
|
||||
|
||||
private static final Option SEED_ROOT_OPTION = Option.builder("s").longOpt("seed-root").hasArg(true).required(true)
|
||||
.desc("path to where seed resource files are stored").build();
|
||||
private static final Options OPTIONS = new Options().addOption(SEED_ROOT_OPTION);
|
||||
|
||||
private final GcsBucketSpecFetcher bucketSpecFetcher;
|
||||
|
||||
public SeedConnectorSpecGenerator(final GcsBucketSpecFetcher bucketSpecFetcher) {
|
||||
this.bucketSpecFetcher = bucketSpecFetcher;
|
||||
}
|
||||
|
||||
public static void main(final String[] args) throws Exception {
|
||||
final CommandLine parsed = Clis.parse(args, OPTIONS);
|
||||
final Path outputRoot = Path.of(parsed.getOptionValue(SEED_ROOT_OPTION.getOpt()));
|
||||
|
||||
final GcsBucketSpecFetcher bucketSpecFetcher = new GcsBucketSpecFetcher(StorageOptions.getDefaultInstance().getService(), SPEC_BUCKET_NAME);
|
||||
final SeedConnectorSpecGenerator seedConnectorSpecGenerator = new SeedConnectorSpecGenerator(bucketSpecFetcher);
|
||||
seedConnectorSpecGenerator.run(outputRoot, SeedConnectorType.SOURCE);
|
||||
seedConnectorSpecGenerator.run(outputRoot, SeedConnectorType.DESTINATION);
|
||||
}
|
||||
|
||||
public void run(final Path seedRoot, final SeedConnectorType seedConnectorType) throws IOException {
|
||||
LOGGER.info("Updating seeded {} definition specs if necessary...", seedConnectorType.name());
|
||||
|
||||
final JsonNode seedDefinitionsJson = yamlToJson(seedRoot, seedConnectorType.getDefinitionFileName());
|
||||
final JsonNode seedSpecsJson = yamlToJson(seedRoot, seedConnectorType.getSpecFileName());
|
||||
|
||||
final List<DockerImageSpec> updatedSeedSpecs = fetchUpdatedSeedSpecs(seedDefinitionsJson, seedSpecsJson);
|
||||
|
||||
final String outputString = String.format("# This file is generated by %s.\n", this.getClass().getName())
|
||||
+ "# Do NOT edit this file directly. See generator class for more details.\n"
|
||||
+ Yamls.serialize(updatedSeedSpecs);
|
||||
final Path outputPath = IOs.writeFile(seedRoot.resolve(seedConnectorType.getSpecFileName()), outputString);
|
||||
|
||||
LOGGER.info("Finished updating {}", outputPath);
|
||||
}
|
||||
|
||||
private JsonNode yamlToJson(final Path root, final String fileName) {
|
||||
final String yamlString = IOs.readFile(root, fileName);
|
||||
return Yamls.deserialize(yamlString);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
final List<DockerImageSpec> fetchUpdatedSeedSpecs(final JsonNode seedDefinitions, final JsonNode currentSeedSpecs) {
|
||||
final List<String> seedDefinitionsDockerImages = MoreIterators.toList(seedDefinitions.elements())
|
||||
.stream()
|
||||
.map(json -> String.format("%s:%s", json.get(DOCKER_REPOSITORY_FIELD).asText(), json.get(DOCKER_IMAGE_TAG_FIELD).asText()))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
final Map<String, DockerImageSpec> currentSeedImageToSpec = MoreIterators.toList(currentSeedSpecs.elements())
|
||||
.stream()
|
||||
.collect(Collectors.toMap(
|
||||
json -> json.get(DOCKER_IMAGE_FIELD).asText(),
|
||||
json -> new DockerImageSpec().withDockerImage(json.get(DOCKER_IMAGE_FIELD).asText())
|
||||
.withSpec(Jsons.object(json.get(SPEC_FIELD), ConnectorSpecification.class))));
|
||||
|
||||
return seedDefinitionsDockerImages
|
||||
.stream()
|
||||
.map(dockerImage -> currentSeedImageToSpec.containsKey(dockerImage) ? currentSeedImageToSpec.get(dockerImage) : fetchSpecFromGCS(dockerImage))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private DockerImageSpec fetchSpecFromGCS(final String dockerImage) {
|
||||
LOGGER.info("Seeded spec not found for docker image {} - fetching from GCS bucket {}...", dockerImage, bucketSpecFetcher.getBucketName());
|
||||
final ConnectorSpecification spec = bucketSpecFetcher.attemptFetch(dockerImage)
|
||||
.orElseThrow(() -> new RuntimeException(String.format(
|
||||
"Failed to fetch valid spec file for docker image %s from GCS bucket %s",
|
||||
dockerImage,
|
||||
bucketSpecFetcher.getBucketName())));
|
||||
return new DockerImageSpec().withDockerImage(dockerImage).withSpec(spec);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -1,33 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2021 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.config.specs;
|
||||
|
||||
public enum SeedConnectorType {
|
||||
|
||||
SOURCE(
|
||||
"source_definitions.yaml",
|
||||
"source_specs.yaml"),
|
||||
DESTINATION(
|
||||
"destination_definitions.yaml",
|
||||
"destination_specs.yaml");
|
||||
|
||||
private final String definitionFileName;
|
||||
private final String specFileName;
|
||||
|
||||
SeedConnectorType(final String definitionFileName,
|
||||
final String specFileName) {
|
||||
this.definitionFileName = definitionFileName;
|
||||
this.specFileName = specFileName;
|
||||
}
|
||||
|
||||
public String getDefinitionFileName() {
|
||||
return definitionFileName;
|
||||
}
|
||||
|
||||
public String getSpecFileName() {
|
||||
return specFileName;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -1,79 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2021 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.config.specs;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import com.google.cloud.storage.Blob;
|
||||
import com.google.cloud.storage.Storage;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.protocol.models.ConnectorSpecification;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Optional;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
class GcsBucketSpecFetcherTest {
|
||||
|
||||
private static final String BUCKET_NAME = "bucket";
|
||||
private static final String DOCKER_REPOSITORY = "image";
|
||||
private static final String DOCKER_IMAGE_TAG = "0.1.0";
|
||||
private static final String DOCKER_IMAGE = DOCKER_REPOSITORY + ":" + DOCKER_IMAGE_TAG;
|
||||
private static final String SPEC_PATH = Path.of("specs").resolve(DOCKER_REPOSITORY).resolve(DOCKER_IMAGE_TAG).resolve("spec.json").toString();
|
||||
|
||||
private Storage storage;
|
||||
private Blob specBlob;
|
||||
private final ConnectorSpecification spec = new ConnectorSpecification().withConnectionSpecification(Jsons.jsonNode(ImmutableMap.of("foo", "bar")));
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@BeforeEach
|
||||
void setup() throws IOException {
|
||||
storage = mock(Storage.class);
|
||||
|
||||
final byte[] specBytes = Jsons.toBytes(Jsons.jsonNode(spec));
|
||||
specBlob = mock(Blob.class);
|
||||
when(specBlob.getContent()).thenReturn(specBytes);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGetsSpecIfPresent() throws IOException {
|
||||
when(storage.get(BUCKET_NAME, SPEC_PATH)).thenReturn(specBlob);
|
||||
|
||||
final GcsBucketSpecFetcher bucketSpecFetcher = new GcsBucketSpecFetcher(storage, BUCKET_NAME);
|
||||
final Optional<ConnectorSpecification> returnedSpec = bucketSpecFetcher.attemptFetch(DOCKER_IMAGE);
|
||||
|
||||
assertTrue(returnedSpec.isPresent());
|
||||
assertEquals(spec, returnedSpec.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReturnsEmptyIfNotPresent() throws IOException {
|
||||
when(storage.get(BUCKET_NAME, SPEC_PATH)).thenReturn(null);
|
||||
|
||||
final GcsBucketSpecFetcher bucketSpecFetcher = new GcsBucketSpecFetcher(storage, BUCKET_NAME);
|
||||
final Optional<ConnectorSpecification> returnedSpec = bucketSpecFetcher.attemptFetch(DOCKER_IMAGE);
|
||||
|
||||
assertTrue(returnedSpec.isEmpty());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReturnsEmptyIfInvalidSpec() throws IOException {
|
||||
final Blob invalidSpecBlob = mock(Blob.class);
|
||||
when(invalidSpecBlob.getContent()).thenReturn("{\"notASpec\": true}".getBytes(StandardCharsets.UTF_8));
|
||||
when(storage.get(BUCKET_NAME, SPEC_PATH)).thenReturn(invalidSpecBlob);
|
||||
|
||||
final GcsBucketSpecFetcher bucketSpecFetcher = new GcsBucketSpecFetcher(storage, BUCKET_NAME);
|
||||
final Optional<ConnectorSpecification> returnedSpec = bucketSpecFetcher.attemptFetch(DOCKER_IMAGE);
|
||||
|
||||
assertTrue(returnedSpec.isEmpty());
|
||||
}
|
||||
|
||||
}
|
||||
@@ -1,154 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2021 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.config.specs;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.never;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.config.DockerImageSpec;
|
||||
import io.airbyte.config.StandardDestinationDefinition;
|
||||
import io.airbyte.protocol.models.ConnectorSpecification;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.UUID;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
class SeedConnectorSpecGeneratorTest {
|
||||
|
||||
private static final UUID DEF_ID1 = java.util.UUID.randomUUID();
|
||||
private static final UUID DEF_ID2 = java.util.UUID.randomUUID();
|
||||
private static final String CONNECTOR_NAME1 = "connector1";
|
||||
private static final String CONNECTOR_NAME2 = "connector2";
|
||||
private static final String DOCUMENTATION_URL = "https://wwww.example.com";
|
||||
private static final String DOCKER_REPOSITORY1 = "airbyte/connector1";
|
||||
private static final String DOCKER_REPOSITORY2 = "airbyte/connector2";
|
||||
private static final String DOCKER_TAG1 = "0.1.0";
|
||||
private static final String DOCKER_TAG2 = "0.2.0";
|
||||
private static final String BUCKET_NAME = "bucket";
|
||||
|
||||
private SeedConnectorSpecGenerator seedConnectorSpecGenerator;
|
||||
private GcsBucketSpecFetcher bucketSpecFetcherMock;
|
||||
|
||||
@BeforeEach
|
||||
void setup() {
|
||||
bucketSpecFetcherMock = mock(GcsBucketSpecFetcher.class);
|
||||
when(bucketSpecFetcherMock.getBucketName()).thenReturn(BUCKET_NAME);
|
||||
|
||||
seedConnectorSpecGenerator = new SeedConnectorSpecGenerator(bucketSpecFetcherMock);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testMissingSpecIsFetched() {
|
||||
final StandardDestinationDefinition sourceDefinition1 = new StandardDestinationDefinition()
|
||||
.withDestinationDefinitionId(DEF_ID1)
|
||||
.withDockerRepository(DOCKER_REPOSITORY1)
|
||||
.withDockerImageTag(DOCKER_TAG1)
|
||||
.withName(CONNECTOR_NAME1)
|
||||
.withDocumentationUrl(DOCUMENTATION_URL);
|
||||
final ConnectorSpecification spec1 = new ConnectorSpecification().withConnectionSpecification(Jsons.jsonNode(ImmutableMap.of("foo1", "bar1")));
|
||||
final DockerImageSpec dockerImageSpec1 = new DockerImageSpec().withDockerImage(DOCKER_REPOSITORY1 + ":" + DOCKER_TAG1).withSpec(spec1);
|
||||
|
||||
final StandardDestinationDefinition sourceDefinition2 = new StandardDestinationDefinition()
|
||||
.withDestinationDefinitionId(DEF_ID2)
|
||||
.withDockerRepository(DOCKER_REPOSITORY2)
|
||||
.withDockerImageTag(DOCKER_TAG2)
|
||||
.withName(CONNECTOR_NAME2)
|
||||
.withDocumentationUrl(DOCUMENTATION_URL);
|
||||
final ConnectorSpecification spec2 = new ConnectorSpecification().withConnectionSpecification(Jsons.jsonNode(ImmutableMap.of("foo2", "bar2")));
|
||||
final DockerImageSpec dockerImageSpec2 = new DockerImageSpec().withDockerImage(DOCKER_REPOSITORY2 + ":" + DOCKER_TAG2).withSpec(spec2);
|
||||
|
||||
final JsonNode seedDefinitions = Jsons.jsonNode(Arrays.asList(sourceDefinition1, sourceDefinition2));
|
||||
final JsonNode seedSpecs = Jsons.jsonNode(List.of(dockerImageSpec1));
|
||||
|
||||
when(bucketSpecFetcherMock.attemptFetch(DOCKER_REPOSITORY2 + ":" + DOCKER_TAG2)).thenReturn(Optional.of(spec2));
|
||||
|
||||
final List<DockerImageSpec> actualSeedSpecs = seedConnectorSpecGenerator.fetchUpdatedSeedSpecs(seedDefinitions, seedSpecs);
|
||||
final List<DockerImageSpec> expectedSeedSpecs = Arrays.asList(dockerImageSpec1, dockerImageSpec2);
|
||||
|
||||
assertEquals(expectedSeedSpecs, actualSeedSpecs);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testOutdatedSpecIsFetched() {
|
||||
final StandardDestinationDefinition sourceDefinition = new StandardDestinationDefinition()
|
||||
.withDestinationDefinitionId(DEF_ID1)
|
||||
.withDockerRepository(DOCKER_REPOSITORY1)
|
||||
.withDockerImageTag(DOCKER_TAG2)
|
||||
.withName(CONNECTOR_NAME1)
|
||||
.withDocumentationUrl(DOCUMENTATION_URL);
|
||||
final ConnectorSpecification outdatedSpec = new ConnectorSpecification().withConnectionSpecification(Jsons.jsonNode(ImmutableMap.of(
|
||||
"foo1",
|
||||
"bar1")));
|
||||
final DockerImageSpec outdatedDockerImageSpec = new DockerImageSpec().withDockerImage(DOCKER_REPOSITORY1 + ":" + DOCKER_TAG1)
|
||||
.withSpec(outdatedSpec);
|
||||
|
||||
final JsonNode seedDefinitions = Jsons.jsonNode(List.of(sourceDefinition));
|
||||
final JsonNode seedSpecs = Jsons.jsonNode(List.of(outdatedDockerImageSpec));
|
||||
|
||||
final ConnectorSpecification newSpec = new ConnectorSpecification().withConnectionSpecification(Jsons.jsonNode(ImmutableMap.of("foo2", "bar2")));
|
||||
final DockerImageSpec newDockerImageSpec = new DockerImageSpec().withDockerImage(DOCKER_REPOSITORY1 + ":" + DOCKER_TAG2).withSpec(newSpec);
|
||||
|
||||
when(bucketSpecFetcherMock.attemptFetch(DOCKER_REPOSITORY1 + ":" + DOCKER_TAG2)).thenReturn(Optional.of(newSpec));
|
||||
|
||||
final List<DockerImageSpec> actualSeedSpecs = seedConnectorSpecGenerator.fetchUpdatedSeedSpecs(seedDefinitions, seedSpecs);
|
||||
final List<DockerImageSpec> expectedSeedSpecs = List.of(newDockerImageSpec);
|
||||
|
||||
assertEquals(expectedSeedSpecs, actualSeedSpecs);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testExtraneousSpecIsRemoved() {
|
||||
final StandardDestinationDefinition sourceDefinition = new StandardDestinationDefinition()
|
||||
.withDestinationDefinitionId(DEF_ID1)
|
||||
.withDockerRepository(DOCKER_REPOSITORY1)
|
||||
.withDockerImageTag(DOCKER_TAG1)
|
||||
.withName(CONNECTOR_NAME1)
|
||||
.withDocumentationUrl(DOCUMENTATION_URL);
|
||||
final ConnectorSpecification spec1 = new ConnectorSpecification().withConnectionSpecification(Jsons.jsonNode(ImmutableMap.of("foo1", "bar1")));
|
||||
final DockerImageSpec dockerImageSpec1 = new DockerImageSpec().withDockerImage(DOCKER_REPOSITORY1 + ":" + DOCKER_TAG1).withSpec(spec1);
|
||||
|
||||
final ConnectorSpecification spec2 = new ConnectorSpecification().withConnectionSpecification(Jsons.jsonNode(ImmutableMap.of("foo2", "bar2")));
|
||||
final DockerImageSpec dockerImageSpec2 = new DockerImageSpec().withDockerImage(DOCKER_REPOSITORY2 + ":" + DOCKER_TAG2).withSpec(spec2);
|
||||
|
||||
final JsonNode seedDefinitions = Jsons.jsonNode(List.of(sourceDefinition));
|
||||
final JsonNode seedSpecs = Jsons.jsonNode(Arrays.asList(dockerImageSpec1, dockerImageSpec2));
|
||||
|
||||
final List<DockerImageSpec> actualSeedSpecs = seedConnectorSpecGenerator.fetchUpdatedSeedSpecs(seedDefinitions, seedSpecs);
|
||||
final List<DockerImageSpec> expectedSeedSpecs = List.of(dockerImageSpec1);
|
||||
|
||||
assertEquals(expectedSeedSpecs, actualSeedSpecs);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testNoFetchIsPerformedIfAllSpecsUpToDate() {
|
||||
final StandardDestinationDefinition sourceDefinition = new StandardDestinationDefinition()
|
||||
.withDestinationDefinitionId(DEF_ID1)
|
||||
.withDockerRepository(DOCKER_REPOSITORY1)
|
||||
.withDockerImageTag(DOCKER_TAG1)
|
||||
.withName(CONNECTOR_NAME1)
|
||||
.withDocumentationUrl(DOCUMENTATION_URL);
|
||||
final ConnectorSpecification spec = new ConnectorSpecification().withConnectionSpecification(Jsons.jsonNode(ImmutableMap.of("foo", "bar")));
|
||||
final DockerImageSpec dockerImageSpec = new DockerImageSpec().withDockerImage(DOCKER_REPOSITORY1 + ":" + DOCKER_TAG1).withSpec(spec);
|
||||
|
||||
final JsonNode seedDefinitions = Jsons.jsonNode(List.of(sourceDefinition));
|
||||
final JsonNode seedSpecs = Jsons.jsonNode(List.of(dockerImageSpec));
|
||||
|
||||
final List<DockerImageSpec> actualSeedSpecs = seedConnectorSpecGenerator.fetchUpdatedSeedSpecs(seedDefinitions, seedSpecs);
|
||||
final List<DockerImageSpec> expectedSeedSpecs = List.of(dockerImageSpec);
|
||||
|
||||
assertEquals(expectedSeedSpecs, actualSeedSpecs);
|
||||
verify(bucketSpecFetcherMock, never()).attemptFetch(any());
|
||||
}
|
||||
|
||||
}
|
||||
@@ -6,6 +6,4 @@ dependencies {
|
||||
implementation 'com.networknt:json-schema-validator:1.0.42'
|
||||
// needed so that we can follow $ref when parsing json. jackson does not support this natively.
|
||||
implementation 'me.andrz.jackson:jackson-json-reference-core:0.3.2'
|
||||
|
||||
implementation project(':airbyte-commons')
|
||||
}
|
||||
|
||||
@@ -7,8 +7,6 @@ plugins {
|
||||
dependencies {
|
||||
implementation 'javax.validation:validation-api:1.1.0.Final'
|
||||
implementation 'org.apache.commons:commons-lang3:3.11'
|
||||
|
||||
implementation project(':airbyte-commons')
|
||||
}
|
||||
|
||||
jsonSchema2Pojo {
|
||||
|
||||
@@ -5,7 +5,6 @@ plugins {
|
||||
dependencies {
|
||||
implementation project(':airbyte-config:models')
|
||||
implementation project(':airbyte-config:persistence')
|
||||
implementation project(':airbyte-config:specs')
|
||||
implementation project(':airbyte-json-validation')
|
||||
implementation project(':airbyte-protocol:models')
|
||||
implementation project(':airbyte-scheduler:models')
|
||||
|
||||
@@ -4,17 +4,27 @@
|
||||
|
||||
package io.airbyte.scheduler.client;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.api.client.util.Preconditions;
|
||||
import com.google.cloud.storage.Blob;
|
||||
import com.google.cloud.storage.Storage;
|
||||
import com.google.cloud.storage.StorageOptions;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.config.DestinationConnection;
|
||||
import io.airbyte.config.JobConfig.ConfigType;
|
||||
import io.airbyte.config.SourceConnection;
|
||||
import io.airbyte.config.StandardCheckConnectionOutput;
|
||||
import io.airbyte.config.specs.GcsBucketSpecFetcher;
|
||||
import io.airbyte.protocol.models.AirbyteCatalog;
|
||||
import io.airbyte.protocol.models.AirbyteProtocolSchema;
|
||||
import io.airbyte.protocol.models.ConnectorSpecification;
|
||||
import io.airbyte.validation.json.JsonSchemaValidator;
|
||||
import io.airbyte.validation.json.JsonValidationException;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@@ -23,15 +33,17 @@ public class BucketSpecCacheSchedulerClient implements SynchronousSchedulerClien
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(BucketSpecCacheSchedulerClient.class);
|
||||
|
||||
private final SynchronousSchedulerClient client;
|
||||
private final GcsBucketSpecFetcher bucketSpecFetcher;
|
||||
private final Function<String, Optional<ConnectorSpecification>> bucketSpecFetcher;
|
||||
|
||||
public BucketSpecCacheSchedulerClient(final SynchronousSchedulerClient client, final String bucketName) {
|
||||
this.client = client;
|
||||
this.bucketSpecFetcher = new GcsBucketSpecFetcher(StorageOptions.getDefaultInstance().getService(), bucketName);
|
||||
this(
|
||||
client,
|
||||
dockerImage -> attemptToFetchSpecFromBucket(StorageOptions.getDefaultInstance().getService(), bucketName, dockerImage));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
BucketSpecCacheSchedulerClient(final SynchronousSchedulerClient client, final GcsBucketSpecFetcher bucketSpecFetcher) {
|
||||
BucketSpecCacheSchedulerClient(final SynchronousSchedulerClient client,
|
||||
final Function<String, Optional<ConnectorSpecification>> bucketSpecFetcher) {
|
||||
this.client = client;
|
||||
this.bucketSpecFetcher = bucketSpecFetcher;
|
||||
}
|
||||
@@ -60,7 +72,7 @@ public class BucketSpecCacheSchedulerClient implements SynchronousSchedulerClien
|
||||
Optional<ConnectorSpecification> cachedSpecOptional;
|
||||
// never want to fail because we could not fetch from off board storage.
|
||||
try {
|
||||
cachedSpecOptional = bucketSpecFetcher.attemptFetch(dockerImage);
|
||||
cachedSpecOptional = bucketSpecFetcher.apply(dockerImage);
|
||||
LOGGER.debug("Spec bucket cache: Call to cache did not fail.");
|
||||
} catch (final RuntimeException e) {
|
||||
cachedSpecOptional = Optional.empty();
|
||||
@@ -76,4 +88,38 @@ public class BucketSpecCacheSchedulerClient implements SynchronousSchedulerClien
|
||||
}
|
||||
}
|
||||
|
||||
private static void validateConfig(final JsonNode json) throws JsonValidationException {
|
||||
final JsonSchemaValidator jsonSchemaValidator = new JsonSchemaValidator();
|
||||
final JsonNode specJsonSchema = JsonSchemaValidator.getSchema(AirbyteProtocolSchema.PROTOCOL.getFile(), "ConnectorSpecification");
|
||||
jsonSchemaValidator.ensure(specJsonSchema, json);
|
||||
}
|
||||
|
||||
public static Optional<ConnectorSpecification> attemptToFetchSpecFromBucket(final Storage storage,
|
||||
final String bucketName,
|
||||
final String dockerImage) {
|
||||
final String[] dockerImageComponents = dockerImage.split(":");
|
||||
Preconditions.checkArgument(dockerImageComponents.length == 2, "Invalidate docker image: " + dockerImage);
|
||||
final String dockerImageName = dockerImageComponents[0];
|
||||
final String dockerImageTag = dockerImageComponents[1];
|
||||
|
||||
final Path specPath = Path.of("specs").resolve(dockerImageName).resolve(dockerImageTag).resolve("spec.json");
|
||||
LOGGER.debug("Checking path for cached spec: {} {}", bucketName, specPath);
|
||||
final Blob specAsBlob = storage.get(bucketName, specPath.toString());
|
||||
|
||||
// if null it means the object was not found.
|
||||
if (specAsBlob == null) {
|
||||
LOGGER.debug("Spec not found in bucket storage");
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
final String specAsString = new String(specAsBlob.getContent(), StandardCharsets.UTF_8);
|
||||
try {
|
||||
validateConfig(Jsons.deserialize(specAsString));
|
||||
} catch (final JsonValidationException e) {
|
||||
LOGGER.error("Received invalid spec from bucket store. {}", e.toString());
|
||||
return Optional.empty();
|
||||
}
|
||||
return Optional.of(Jsons.deserialize(specAsString, ConnectorSpecification.class));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -10,10 +10,10 @@ import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.verifyNoInteractions;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import io.airbyte.config.specs.GcsBucketSpecFetcher;
|
||||
import io.airbyte.protocol.models.ConnectorSpecification;
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Disabled;
|
||||
import org.junit.jupiter.api.Test;
|
||||
@@ -21,18 +21,18 @@ import org.junit.jupiter.api.Test;
|
||||
class BucketSpecCacheSchedulerClientTest {
|
||||
|
||||
private SynchronousSchedulerClient defaultClientMock;
|
||||
private GcsBucketSpecFetcher bucketSpecFetcherMock;
|
||||
private Function<String, Optional<ConnectorSpecification>> bucketSpecFetcherMock;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@BeforeEach
|
||||
void setup() {
|
||||
defaultClientMock = mock(SynchronousSchedulerClient.class);
|
||||
bucketSpecFetcherMock = mock(GcsBucketSpecFetcher.class);
|
||||
bucketSpecFetcherMock = mock(Function.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGetsSpecIfPresent() throws IOException {
|
||||
when(bucketSpecFetcherMock.attemptFetch("source-pokeapi:0.1.0")).thenReturn(Optional.of(new ConnectorSpecification()));
|
||||
when(bucketSpecFetcherMock.apply("source-pokeapi:0.1.0")).thenReturn(Optional.of(new ConnectorSpecification()));
|
||||
final BucketSpecCacheSchedulerClient client = new BucketSpecCacheSchedulerClient(defaultClientMock, bucketSpecFetcherMock);
|
||||
assertEquals(new ConnectorSpecification(), client.createGetSpecJob("source-pokeapi:0.1.0").getOutput());
|
||||
verifyNoInteractions(defaultClientMock);
|
||||
@@ -40,7 +40,7 @@ class BucketSpecCacheSchedulerClientTest {
|
||||
|
||||
@Test
|
||||
void testCallsDelegateIfNotPresent() throws IOException {
|
||||
when(bucketSpecFetcherMock.attemptFetch("source-pokeapi:0.1.0")).thenReturn(Optional.empty());
|
||||
when(bucketSpecFetcherMock.apply("source-pokeapi:0.1.0")).thenReturn(Optional.empty());
|
||||
when(defaultClientMock.createGetSpecJob("source-pokeapi:0.1.0"))
|
||||
.thenReturn(new SynchronousResponse<>(new ConnectorSpecification(), mock(SynchronousJobMetadata.class)));
|
||||
final BucketSpecCacheSchedulerClient client = new BucketSpecCacheSchedulerClient(defaultClientMock, bucketSpecFetcherMock);
|
||||
@@ -49,7 +49,7 @@ class BucketSpecCacheSchedulerClientTest {
|
||||
|
||||
@Test
|
||||
void testCallsDelegateIfException() throws IOException {
|
||||
when(bucketSpecFetcherMock.attemptFetch("source-pokeapi:0.1.0")).thenThrow(new RuntimeException("induced exception"));
|
||||
when(bucketSpecFetcherMock.apply("source-pokeapi:0.1.0")).thenThrow(new RuntimeException("induced exception"));
|
||||
when(defaultClientMock.createGetSpecJob("source-pokeapi:0.1.0"))
|
||||
.thenReturn(new SynchronousResponse<>(new ConnectorSpecification(), mock(SynchronousJobMetadata.class)));
|
||||
final BucketSpecCacheSchedulerClient client = new BucketSpecCacheSchedulerClient(defaultClientMock, bucketSpecFetcherMock);
|
||||
@@ -62,7 +62,7 @@ class BucketSpecCacheSchedulerClientTest {
|
||||
@Disabled
|
||||
@Test
|
||||
void testGetsSpecFromBucket() throws IOException {
|
||||
when(bucketSpecFetcherMock.attemptFetch("source-pokeapi:0.1.0")).thenReturn(Optional.of(new ConnectorSpecification()));
|
||||
when(bucketSpecFetcherMock.apply("source-pokeapi:0.1.0")).thenReturn(Optional.of(new ConnectorSpecification()));
|
||||
// todo (cgardens) - replace with prod bucket.
|
||||
final BucketSpecCacheSchedulerClient client = new BucketSpecCacheSchedulerClient(defaultClientMock, "cg-specs");
|
||||
final ConnectorSpecification actualSpec = client.createGetSpecJob("source-pokeapi:0.1.0").getOutput();
|
||||
|
||||
@@ -66,7 +66,6 @@ dependencies {
|
||||
implementation project(':airbyte-config:init')
|
||||
implementation project(':airbyte-config:models')
|
||||
implementation project(':airbyte-config:persistence')
|
||||
implementation project(':airbyte-config:specs')
|
||||
implementation project(':airbyte-db:lib')
|
||||
implementation project(":airbyte-json-validation")
|
||||
implementation project(':airbyte-migration')
|
||||
|
||||
@@ -73,8 +73,7 @@ def createSpotlessTarget = { pattern ->
|
||||
'normalization_test_output',
|
||||
'tools',
|
||||
'secrets',
|
||||
'charts', // Helm charts often have injected template strings that will fail general linting. Helm linting is done separately.
|
||||
'resources/seed/*_specs.yaml'
|
||||
'charts' // Helm charts often have injected template strings that will fail general linting. Helm linting is done separately.
|
||||
]
|
||||
|
||||
if (System.getenv().containsKey("SUB_BUILD")) {
|
||||
|
||||
@@ -42,7 +42,6 @@ include ':airbyte-workers' // reused by acceptance tests in connector base.
|
||||
include ':airbyte-analytics' // transitively used by airbyte-workers.
|
||||
include ':airbyte-config:init' // transitively used by airbyte-workers.
|
||||
include ':airbyte-config:persistence' // transitively used by airbyte-workers.
|
||||
include ':airbyte-config:specs' // transitively used by airbyte-workers.
|
||||
include ':airbyte-db:jooq' // transitively used by airbyte-workers.
|
||||
include ':airbyte-notification' // transitively used by airbyte-workers.
|
||||
include ':airbyte-scheduler:models' // transitively used by airbyte-workers.
|
||||
|
||||
Reference in New Issue
Block a user