diff --git a/airbyte-config/models/src/main/java/io/airbyte/config/Configs.java b/airbyte-config/models/src/main/java/io/airbyte/config/Configs.java index fb16fa848773..7e3c9162a9a5 100644 --- a/airbyte-config/models/src/main/java/io/airbyte/config/Configs.java +++ b/airbyte-config/models/src/main/java/io/airbyte/config/Configs.java @@ -121,8 +121,6 @@ public interface Configs { boolean getPublishMetrics(); - boolean getVersion32ForceUpgrade(); - SecretPersistenceType getSecretPersistenceType(); enum TrackingStrategy { diff --git a/airbyte-config/models/src/main/java/io/airbyte/config/EnvConfigs.java b/airbyte-config/models/src/main/java/io/airbyte/config/EnvConfigs.java index 23135f98843d..8debee38b7a0 100644 --- a/airbyte-config/models/src/main/java/io/airbyte/config/EnvConfigs.java +++ b/airbyte-config/models/src/main/java/io/airbyte/config/EnvConfigs.java @@ -77,7 +77,6 @@ public class EnvConfigs implements Configs { private static final String SECRET_PERSISTENCE = "SECRET_PERSISTENCE"; private static final String JOBS_IMAGE_PULL_SECRET = "JOBS_IMAGE_PULL_SECRET"; private static final String PUBLISH_METRICS = "PUBLISH_METRICS"; - private static final String VERSION_0_32_0_FORCE_UPGRADE = "VERSION_0_32_0_FORCE_UPGRADE"; // defaults private static final String DEFAULT_SPEC_CACHE_BUCKET = "io-airbyte-cloud-spec-cache"; @@ -478,11 +477,6 @@ public boolean getPublishMetrics() { return getEnvOrDefault(PUBLISH_METRICS, false); } - @Override - public boolean getVersion32ForceUpgrade() { - return getEnvOrDefault(VERSION_0_32_0_FORCE_UPGRADE, false); - } - @Override public SecretPersistenceType getSecretPersistenceType() { final var secretPersistenceStr = getEnvOrDefault(SECRET_PERSISTENCE, SecretPersistenceType.NONE.name()); diff --git a/airbyte-config/models/src/main/resources/types/StandardDestinationDefinition.yaml b/airbyte-config/models/src/main/resources/types/StandardDestinationDefinition.yaml index 3abb0482f07a..96bea4098b5b 100644 --- a/airbyte-config/models/src/main/resources/types/StandardDestinationDefinition.yaml +++ b/airbyte-config/models/src/main/resources/types/StandardDestinationDefinition.yaml @@ -10,7 +10,8 @@ required: - dockerRepository - dockerImageTag - documentationUrl -additionalProperties: false + - spec +additionalProperties: true properties: destinationDefinitionId: type: string diff --git a/airbyte-config/models/src/main/resources/types/StandardSourceDefinition.yaml b/airbyte-config/models/src/main/resources/types/StandardSourceDefinition.yaml index 03bd8e2ad2cb..2bb3333e6ea1 100644 --- a/airbyte-config/models/src/main/resources/types/StandardSourceDefinition.yaml +++ b/airbyte-config/models/src/main/resources/types/StandardSourceDefinition.yaml @@ -10,6 +10,7 @@ required: - dockerRepository - dockerImageTag - documentationUrl + - spec additionalProperties: true properties: sourceDefinitionId: diff --git a/airbyte-config/persistence/src/main/java/io/airbyte/config/persistence/ConfigRepository.java b/airbyte-config/persistence/src/main/java/io/airbyte/config/persistence/ConfigRepository.java index bf9ff27c4632..acfec8015ed1 100644 --- a/airbyte-config/persistence/src/main/java/io/airbyte/config/persistence/ConfigRepository.java +++ b/airbyte-config/persistence/src/main/java/io/airbyte/config/persistence/ConfigRepository.java @@ -5,8 +5,6 @@ package io.airbyte.config.persistence; import com.fasterxml.jackson.databind.JsonNode; -import com.google.api.client.util.Preconditions; -import io.airbyte.commons.docker.DockerUtils; import io.airbyte.commons.json.Jsons; import io.airbyte.commons.lang.Exceptions; import io.airbyte.commons.lang.MoreBooleans; @@ -57,7 +55,6 @@ public class ConfigRepository { private final SecretsHydrator secretsHydrator; private final Optional longLivedSecretPersistence; private final Optional ephemeralSecretPersistence; - private Function specFetcherFn; public ConfigRepository(final ConfigPersistence persistence, final SecretsHydrator secretsHydrator, @@ -547,7 +544,6 @@ public void replaceAllConfigsDeserializing(final Map> c public void replaceAllConfigs(final Map> configs, final boolean dryRun) throws IOException { if (longLivedSecretPersistence.isPresent()) { - Preconditions.checkNotNull(specFetcherFn); final var augmentedMap = new HashMap<>(configs); // get all source defs so that we can use their specs when storing secrets. @@ -558,11 +554,7 @@ public void replaceAllConfigs(final Map> configs, final augmentedMap.put(ConfigSchema.STANDARD_SOURCE_DEFINITION, sourceDefs.stream()); final Map sourceDefIdToSpec = sourceDefs .stream() - .collect(Collectors.toMap(StandardSourceDefinition::getSourceDefinitionId, sourceDefinition -> { - final String imageName = DockerUtils - .getTaggedImageName(sourceDefinition.getDockerRepository(), sourceDefinition.getDockerImageTag()); - return specFetcherFn.apply(imageName); - })); + .collect(Collectors.toMap(StandardSourceDefinition::getSourceDefinitionId, StandardSourceDefinition::getSpec)); // get all destination defs so that we can use their specs when storing secrets. @SuppressWarnings("unchecked") @@ -571,11 +563,7 @@ public void replaceAllConfigs(final Map> configs, final augmentedMap.put(ConfigSchema.STANDARD_DESTINATION_DEFINITION, destinationDefs.stream()); final Map destinationDefIdToSpec = destinationDefs .stream() - .collect(Collectors.toMap(StandardDestinationDefinition::getDestinationDefinitionId, destinationDefinition -> { - final String imageName = DockerUtils - .getTaggedImageName(destinationDefinition.getDockerRepository(), destinationDefinition.getDockerImageTag()); - return specFetcherFn.apply(imageName); - })); + .collect(Collectors.toMap(StandardDestinationDefinition::getDestinationDefinitionId, StandardDestinationDefinition::getSpec)); if (augmentedMap.containsKey(ConfigSchema.SOURCE_CONNECTION)) { final Stream augmentedValue = augmentedMap.get(ConfigSchema.SOURCE_CONNECTION) @@ -640,8 +628,4 @@ public void loadData(final ConfigPersistence seedPersistence) throws IOException persistence.loadData(seedPersistence); } - public void setSpecFetcher(final Function specFetcherFn) { - this.specFetcherFn = specFetcherFn; - } - } diff --git a/airbyte-migration/src/main/resources/migrations/migrationV0_14_0/airbyte_config/StandardDestinationDefinition.yaml b/airbyte-migration/src/main/resources/migrations/migrationV0_14_0/airbyte_config/StandardDestinationDefinition.yaml index 103b4fb588c8..eaa0c11e192c 100644 --- a/airbyte-migration/src/main/resources/migrations/migrationV0_14_0/airbyte_config/StandardDestinationDefinition.yaml +++ b/airbyte-migration/src/main/resources/migrations/migrationV0_14_0/airbyte_config/StandardDestinationDefinition.yaml @@ -10,7 +10,7 @@ required: - dockerRepository - dockerImageTag - documentationUrl -additionalProperties: false +additionalProperties: true properties: destinationDefinitionId: type: string diff --git a/airbyte-migration/src/main/resources/migrations/migrationV0_20_0/StandardDestinationDefinition.yaml b/airbyte-migration/src/main/resources/migrations/migrationV0_20_0/StandardDestinationDefinition.yaml index f58d5af72dde..2a275b303fa3 100644 --- a/airbyte-migration/src/main/resources/migrations/migrationV0_20_0/StandardDestinationDefinition.yaml +++ b/airbyte-migration/src/main/resources/migrations/migrationV0_20_0/StandardDestinationDefinition.yaml @@ -10,7 +10,7 @@ required: - dockerRepository - dockerImageTag - documentationUrl -additionalProperties: false +additionalProperties: true properties: destinationDefinitionId: type: string diff --git a/airbyte-scheduler/client/src/main/java/io/airbyte/scheduler/client/BucketSpecCacheSchedulerClient.java b/airbyte-scheduler/client/src/main/java/io/airbyte/scheduler/client/BucketSpecCacheSchedulerClient.java deleted file mode 100644 index bcdc972c2cb4..000000000000 --- a/airbyte-scheduler/client/src/main/java/io/airbyte/scheduler/client/BucketSpecCacheSchedulerClient.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Copyright (c) 2021 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.scheduler.client; - -import com.google.cloud.storage.StorageOptions; -import com.google.common.annotations.VisibleForTesting; -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.ConnectorSpecification; -import java.io.IOException; -import java.util.Optional; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class BucketSpecCacheSchedulerClient implements SynchronousSchedulerClient { - - private static final Logger LOGGER = LoggerFactory.getLogger(BucketSpecCacheSchedulerClient.class); - - private final SynchronousSchedulerClient client; - private final GcsBucketSpecFetcher bucketSpecFetcher; - - public BucketSpecCacheSchedulerClient(final SynchronousSchedulerClient client, final String bucketName) { - this.client = client; - this.bucketSpecFetcher = new GcsBucketSpecFetcher(StorageOptions.getDefaultInstance().getService(), bucketName); - } - - @VisibleForTesting - BucketSpecCacheSchedulerClient(final SynchronousSchedulerClient client, final GcsBucketSpecFetcher bucketSpecFetcher) { - this.client = client; - this.bucketSpecFetcher = bucketSpecFetcher; - } - - @Override - public SynchronousResponse createSourceCheckConnectionJob(final SourceConnection source, final String dockerImage) - throws IOException { - return client.createSourceCheckConnectionJob(source, dockerImage); - } - - @Override - public SynchronousResponse createDestinationCheckConnectionJob(final DestinationConnection destination, - final String dockerImage) - throws IOException { - return client.createDestinationCheckConnectionJob(destination, dockerImage); - } - - @Override - public SynchronousResponse createDiscoverSchemaJob(final SourceConnection source, final String dockerImage) throws IOException { - return client.createDiscoverSchemaJob(source, dockerImage); - } - - @Override - public SynchronousResponse createGetSpecJob(final String dockerImage) throws IOException { - LOGGER.debug("getting spec!"); - Optional cachedSpecOptional; - // never want to fail because we could not fetch from off board storage. - try { - cachedSpecOptional = bucketSpecFetcher.attemptFetch(dockerImage); - LOGGER.debug("Spec bucket cache: Call to cache did not fail."); - } catch (final RuntimeException e) { - cachedSpecOptional = Optional.empty(); - LOGGER.debug("Spec bucket cache: Call to cache failed."); - } - - if (cachedSpecOptional.isPresent()) { - LOGGER.debug("Spec bucket cache: Cache hit."); - return new SynchronousResponse<>(cachedSpecOptional.get(), SynchronousJobMetadata.mock(ConfigType.GET_SPEC)); - } else { - LOGGER.debug("Spec bucket cache: Cache miss."); - return client.createGetSpecJob(dockerImage); - } - } - -} diff --git a/airbyte-scheduler/client/src/main/java/io/airbyte/scheduler/client/CachingSynchronousSchedulerClient.java b/airbyte-scheduler/client/src/main/java/io/airbyte/scheduler/client/CachingSynchronousSchedulerClient.java deleted file mode 100644 index a2548a48a393..000000000000 --- a/airbyte-scheduler/client/src/main/java/io/airbyte/scheduler/client/CachingSynchronousSchedulerClient.java +++ /dev/null @@ -1,15 +0,0 @@ -/* - * Copyright (c) 2021 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.scheduler.client; - -/** - * This client is meant to be an interface over a cached implementation of - * {@link SynchronousSchedulerClient}. It exposes functionality to allow invalidating the cache. - */ -public interface CachingSynchronousSchedulerClient extends SynchronousSchedulerClient { - - void resetCache(); - -} diff --git a/airbyte-scheduler/client/src/main/java/io/airbyte/scheduler/client/SpecCachingSynchronousSchedulerClient.java b/airbyte-scheduler/client/src/main/java/io/airbyte/scheduler/client/SpecCachingSynchronousSchedulerClient.java deleted file mode 100644 index 555a7cf62c71..000000000000 --- a/airbyte-scheduler/client/src/main/java/io/airbyte/scheduler/client/SpecCachingSynchronousSchedulerClient.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Copyright (c) 2021 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.scheduler.client; - -import com.google.common.cache.Cache; -import com.google.common.cache.CacheBuilder; -import io.airbyte.config.DestinationConnection; -import io.airbyte.config.SourceConnection; -import io.airbyte.config.StandardCheckConnectionOutput; -import io.airbyte.protocol.models.AirbyteCatalog; -import io.airbyte.protocol.models.ConnectorSpecification; -import java.io.IOException; -import java.util.Optional; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This client caches only the results of spec jobs as their output should not change (except in the - * case where the docker image is replaced with an image of the same name and tag) and they are - * called very frequently. - */ -public class SpecCachingSynchronousSchedulerClient implements CachingSynchronousSchedulerClient { - - private static final Logger LOGGER = LoggerFactory.getLogger(SpecCachingSynchronousSchedulerClient.class); - - private final Cache> specCache; - private final SynchronousSchedulerClient decoratedClient; - - public SpecCachingSynchronousSchedulerClient(final SynchronousSchedulerClient decoratedClient) { - this.decoratedClient = decoratedClient; - this.specCache = CacheBuilder.newBuilder().build(); - } - - @Override - public SynchronousResponse createSourceCheckConnectionJob(final SourceConnection source, final String dockerImage) - throws IOException { - return decoratedClient.createSourceCheckConnectionJob(source, dockerImage); - } - - @Override - public SynchronousResponse createDestinationCheckConnectionJob(final DestinationConnection destination, - final String dockerImage) - throws IOException { - return decoratedClient.createDestinationCheckConnectionJob(destination, dockerImage); - } - - @Override - public SynchronousResponse createDiscoverSchemaJob(final SourceConnection source, final String dockerImage) throws IOException { - return decoratedClient.createDiscoverSchemaJob(source, dockerImage); - } - - @Override - public SynchronousResponse createGetSpecJob(final String dockerImage) throws IOException { - final Optional> cachedJob = Optional.ofNullable(specCache.getIfPresent(dockerImage)); - if (cachedJob.isPresent()) { - LOGGER.debug("cache hit: " + dockerImage); - return cachedJob.get(); - } else { - LOGGER.debug("cache miss: " + dockerImage); - final SynchronousResponse response = decoratedClient.createGetSpecJob(dockerImage); - if (response.isSuccess()) { - specCache.put(dockerImage, response); - } - return response; - } - } - - @Override - public void resetCache() { - specCache.invalidateAll(); - } - -} diff --git a/airbyte-scheduler/client/src/test/java/io/airbyte/scheduler/client/BucketSpecCacheSchedulerClientTest.java b/airbyte-scheduler/client/src/test/java/io/airbyte/scheduler/client/BucketSpecCacheSchedulerClientTest.java deleted file mode 100644 index 01f4595b9468..000000000000 --- a/airbyte-scheduler/client/src/test/java/io/airbyte/scheduler/client/BucketSpecCacheSchedulerClientTest.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Copyright (c) 2021 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.scheduler.client; - -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.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 org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Test; - -class BucketSpecCacheSchedulerClientTest { - - private SynchronousSchedulerClient defaultClientMock; - private GcsBucketSpecFetcher bucketSpecFetcherMock; - - @SuppressWarnings("unchecked") - @BeforeEach - void setup() { - defaultClientMock = mock(SynchronousSchedulerClient.class); - bucketSpecFetcherMock = mock(GcsBucketSpecFetcher.class); - } - - @Test - void testGetsSpecIfPresent() throws IOException { - when(bucketSpecFetcherMock.attemptFetch("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); - } - - @Test - void testCallsDelegateIfNotPresent() throws IOException { - when(bucketSpecFetcherMock.attemptFetch("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); - assertEquals(new ConnectorSpecification(), client.createGetSpecJob("source-pokeapi:0.1.0").getOutput()); - } - - @Test - void testCallsDelegateIfException() throws IOException { - when(bucketSpecFetcherMock.attemptFetch("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); - assertEquals(new ConnectorSpecification(), client.createGetSpecJob("source-pokeapi:0.1.0").getOutput()); - } - - // todo (cgardens) - this is essentially an integration test. run it manually to sanity check that - // the client can pull. from the spec cache bucket. when we have a better setup for integation - // testing for the platform we should move it there. - @Disabled - @Test - void testGetsSpecFromBucket() throws IOException { - when(bucketSpecFetcherMock.attemptFetch("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(); - assertTrue(actualSpec.getDocumentationUrl().toString().contains("poke")); - } - -} diff --git a/airbyte-scheduler/client/src/test/java/io/airbyte/scheduler/client/SpecCachingSynchronousSchedulerClientTest.java b/airbyte-scheduler/client/src/test/java/io/airbyte/scheduler/client/SpecCachingSynchronousSchedulerClientTest.java deleted file mode 100644 index 96da5d0cb898..000000000000 --- a/airbyte-scheduler/client/src/test/java/io/airbyte/scheduler/client/SpecCachingSynchronousSchedulerClientTest.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Copyright (c) 2021 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.scheduler.client; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.mockito.Mockito.RETURNS_DEEP_STUBS; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import io.airbyte.protocol.models.ConnectorSpecification; -import java.io.IOException; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -class SpecCachingSynchronousSchedulerClientTest { - - private static final String DOCKER_IMAGE = "airbyte/space_cop"; - - private SynchronousSchedulerClient decoratedClient; - private CachingSynchronousSchedulerClient client; - private SynchronousResponse response1; - private SynchronousResponse response2; - - @SuppressWarnings("unchecked") - @BeforeEach - void setup() { - response1 = mock(SynchronousResponse.class, RETURNS_DEEP_STUBS); - response2 = mock(SynchronousResponse.class, RETURNS_DEEP_STUBS); - decoratedClient = mock(SynchronousSchedulerClient.class); - client = new SpecCachingSynchronousSchedulerClient(decoratedClient); - } - - @Test - void testCreateGetSpecJobCacheCacheMiss() throws IOException { - when(decoratedClient.createGetSpecJob(DOCKER_IMAGE)).thenReturn(response1); - when(response1.isSuccess()).thenReturn(true); - assertEquals(response1, client.createGetSpecJob(DOCKER_IMAGE)); - verify(decoratedClient, times(1)).createGetSpecJob(DOCKER_IMAGE); - } - - @Test - void testCreateGetSpecJobFails() throws IOException { - when(decoratedClient.createGetSpecJob(DOCKER_IMAGE)).thenReturn(response1).thenReturn(response2); - when(response1.isSuccess()).thenReturn(false); - when(response2.isSuccess()).thenReturn(true); - client.createGetSpecJob(DOCKER_IMAGE); - assertEquals(response2, client.createGetSpecJob(DOCKER_IMAGE)); - verify(decoratedClient, times(2)).createGetSpecJob(DOCKER_IMAGE); - } - - @Test - void testCreateGetSpecJobCacheCacheHit() throws IOException { - when(decoratedClient.createGetSpecJob(DOCKER_IMAGE)).thenReturn(response1); - when(response1.isSuccess()).thenReturn(true); - client.createGetSpecJob(DOCKER_IMAGE); - assertEquals(response1, client.createGetSpecJob(DOCKER_IMAGE)); - verify(decoratedClient, times(1)).createGetSpecJob(DOCKER_IMAGE); - } - - @Test - void testInvalidateCache() throws IOException { - when(decoratedClient.createGetSpecJob(DOCKER_IMAGE)).thenReturn(response1).thenReturn(response2); - when(response1.isSuccess()).thenReturn(true); - when(response2.isSuccess()).thenReturn(true); - client.createGetSpecJob(DOCKER_IMAGE); - client.resetCache(); - assertEquals(response2, client.createGetSpecJob(DOCKER_IMAGE)); - verify(decoratedClient, times(2)).createGetSpecJob(DOCKER_IMAGE); - } - -} diff --git a/airbyte-server/src/main/java/io/airbyte/server/ConfigDumpImporter.java b/airbyte-server/src/main/java/io/airbyte/server/ConfigDumpImporter.java index de7ac4084d47..d4e268d284c3 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/ConfigDumpImporter.java +++ b/airbyte-server/src/main/java/io/airbyte/server/ConfigDumpImporter.java @@ -32,10 +32,7 @@ import io.airbyte.scheduler.persistence.DefaultJobPersistence; import io.airbyte.scheduler.persistence.JobPersistence; import io.airbyte.scheduler.persistence.WorkspaceHelper; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.server.errors.IdNotFoundKnownException; -import io.airbyte.server.handlers.DestinationHandler; -import io.airbyte.server.handlers.SourceHandler; import io.airbyte.validation.json.JsonSchemaValidator; import io.airbyte.validation.json.JsonValidationException; import java.io.File; @@ -73,7 +70,6 @@ public class ConfigDumpImporter { private final ConfigRepository configRepository; private final WorkspaceHelper workspaceHelper; - private final SpecFetcher specFetcher; private final JsonSchemaValidator jsonSchemaValidator; private final JobPersistence jobPersistence; private final boolean importDefinitions; @@ -81,9 +77,8 @@ public class ConfigDumpImporter { public ConfigDumpImporter(final ConfigRepository configRepository, final JobPersistence jobPersistence, final WorkspaceHelper workspaceHelper, - final SpecFetcher specFetcher, final boolean importDefinitions) { - this(configRepository, jobPersistence, workspaceHelper, new JsonSchemaValidator(), specFetcher, importDefinitions); + this(configRepository, jobPersistence, workspaceHelper, new JsonSchemaValidator(), importDefinitions); } @VisibleForTesting @@ -91,13 +86,11 @@ public ConfigDumpImporter(final ConfigRepository configRepository, final JobPersistence jobPersistence, final WorkspaceHelper workspaceHelper, final JsonSchemaValidator jsonSchemaValidator, - final SpecFetcher specFetcher, final boolean importDefinitions) { this.jsonSchemaValidator = jsonSchemaValidator; this.jobPersistence = jobPersistence; this.configRepository = configRepository; this.workspaceHelper = workspaceHelper; - this.specFetcher = specFetcher; this.importDefinitions = importDefinitions; } @@ -420,7 +413,7 @@ private void importConfigsIntoWorkspace(final Path sourceRoot, final UUID wo if (sourceDefinition == null) { return; } - configRepository.writeSourceConnection(sourceConnection, SourceHandler.getSpecFromSourceDefinitionId(specFetcher, sourceDefinition)); + configRepository.writeSourceConnection(sourceConnection, sourceDefinition.getSpec()); } catch (final ConfigNotFoundException e) { return; } @@ -448,7 +441,7 @@ private void importConfigsIntoWorkspace(final Path sourceRoot, final UUID wo if (destinationDefinition == null) { return; } - configRepository.writeDestinationConnection(destinationConnection, DestinationHandler.getSpec(specFetcher, destinationDefinition)); + configRepository.writeDestinationConnection(destinationConnection, destinationDefinition.getSpec()); } catch (final ConfigNotFoundException e) { return; } diff --git a/airbyte-server/src/main/java/io/airbyte/server/ConfigurationApiFactory.java b/airbyte-server/src/main/java/io/airbyte/server/ConfigurationApiFactory.java index bdc9dd0105f4..4ba56a22b9c4 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/ConfigurationApiFactory.java +++ b/airbyte-server/src/main/java/io/airbyte/server/ConfigurationApiFactory.java @@ -12,8 +12,8 @@ import io.airbyte.config.persistence.ConfigPersistence; import io.airbyte.config.persistence.ConfigRepository; import io.airbyte.db.Database; -import io.airbyte.scheduler.client.CachingSynchronousSchedulerClient; import io.airbyte.scheduler.client.SchedulerJobClient; +import io.airbyte.scheduler.client.SynchronousSchedulerClient; import io.airbyte.scheduler.persistence.JobPersistence; import io.airbyte.server.apis.ConfigurationApi; import io.temporal.serviceclient.WorkflowServiceStubs; @@ -30,7 +30,7 @@ public class ConfigurationApiFactory implements Factory { private static JobPersistence jobPersistence; private static ConfigPersistence seed; private static SchedulerJobClient schedulerJobClient; - private static CachingSynchronousSchedulerClient synchronousSchedulerClient; + private static SynchronousSchedulerClient synchronousSchedulerClient; private static FileTtlManager archiveTtlManager; private static Map mdc; private static Database configsDatabase; @@ -49,7 +49,7 @@ public static void setValues( final JobPersistence jobPersistence, final ConfigPersistence seed, final SchedulerJobClient schedulerJobClient, - final CachingSynchronousSchedulerClient synchronousSchedulerClient, + final SynchronousSchedulerClient synchronousSchedulerClient, final FileTtlManager archiveTtlManager, final Map mdc, final Database configsDatabase, diff --git a/airbyte-server/src/main/java/io/airbyte/server/ConnectorDefinitionSpecBackfiller.java b/airbyte-server/src/main/java/io/airbyte/server/ConnectorDefinitionSpecBackfiller.java deleted file mode 100644 index 302e751ee4a9..000000000000 --- a/airbyte-server/src/main/java/io/airbyte/server/ConnectorDefinitionSpecBackfiller.java +++ /dev/null @@ -1,226 +0,0 @@ -/* - * Copyright (c) 2021 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.server; - -import com.google.api.client.util.Lists; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableMap; -import io.airbyte.analytics.TrackingClient; -import io.airbyte.api.model.LogRead; -import io.airbyte.commons.json.Jsons; -import io.airbyte.commons.resources.MoreResources; -import io.airbyte.commons.version.AirbyteVersion; -import io.airbyte.config.ConfigSchema; -import io.airbyte.config.Configs; -import io.airbyte.config.StandardDestinationDefinition; -import io.airbyte.config.StandardSourceDefinition; -import io.airbyte.config.persistence.ConfigNotFoundException; -import io.airbyte.config.persistence.ConfigPersistence; -import io.airbyte.config.persistence.ConfigRepository; -import io.airbyte.config.persistence.DatabaseConfigPersistence; -import io.airbyte.protocol.models.ConnectorSpecification; -import io.airbyte.scheduler.client.SynchronousResponse; -import io.airbyte.scheduler.client.SynchronousSchedulerClient; -import io.airbyte.server.converters.JobConverter; -import io.airbyte.validation.json.JsonValidationException; -import java.io.IOException; -import java.util.List; -import java.util.Set; -import java.util.UUID; -import java.util.stream.Collectors; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -// todo (lmossman) - delete this class after the faux major version bump, along with making the spec -// field required on the definition structs -public class ConnectorDefinitionSpecBackfiller { - - private static final Logger LOGGER = LoggerFactory.getLogger(ConnectorDefinitionSpecBackfiller.class); - private static final AirbyteVersion VERSION_BREAK = new AirbyteVersion("0.32.0-alpha"); - - /** - * Check that each spec in the database has a spec. If it doesn't, add it. If it can't be added, - * delete the connector definition or fail according to the VERSION_0_32_0_FORCE_UPGRADE env var. - * The goal is to try to end up in a state where all definitions in the db contain specs, or fail - * otherwise. - * - * @param configRepository - access to the db - * @param database - access to the db at a lower level (for fetching in use docker images) - * @param schedulerClient - scheduler client so that specs can be fetched as needed - * @param trackingClient - tracking client for reporting failures to Segment - * @param configs - for retrieving various configs (env vars, worker environment, logs) - */ - @VisibleForTesting - static void migrateAllDefinitionsToContainSpec(final ConfigRepository configRepository, - final DatabaseConfigPersistence database, - final ConfigPersistence seed, - final SynchronousSchedulerClient schedulerClient, - final TrackingClient trackingClient, - final Configs configs) - throws IOException, JsonValidationException, ConfigNotFoundException { - final List failedBackfillImages = Lists.newArrayList(); - - final JobConverter jobConverter = new JobConverter(configs.getWorkerEnvironment(), configs.getLogConfigs()); - - final Set connectorReposInUse = database.getInUseConnectorDockerImageNames(); - - final Set seedSourceRepos = seed.listConfigs(ConfigSchema.STANDARD_SOURCE_DEFINITION, StandardSourceDefinition.class) - .stream() - .map(StandardSourceDefinition::getDockerRepository) - .collect(Collectors.toSet()); - for (final StandardSourceDefinition sourceDef : configRepository.listStandardSourceDefinitions()) { - // If source definition already has a spec, there is nothing to backfill - if (sourceDef.getSpec() != null) { - continue; - } - - final String imageName = sourceDef.getDockerRepository() + ":" + sourceDef.getDockerImageTag(); - - // if a source definition is not being used and is not in the seed, don't bother to attempt to fetch - // a spec for it; just delete. - if (!connectorReposInUse.contains(sourceDef.getDockerRepository()) && !seedSourceRepos.contains(sourceDef.getDockerRepository())) { - LOGGER.info( - "migrateAllDefinitionsToContainSpec - Source Definition {} does not have a spec, is not in the seed, and is not currently used in a connection. Deleting...", - sourceDef.getName()); - configRepository.deleteSourceDefinitionAndAssociations(sourceDef.getSourceDefinitionId()); - continue; - } - - LOGGER.info( - "migrateAllDefinitionsToContainSpec - Source Definition {} does not have a spec. Attempting to retrieve spec...", - sourceDef.getName()); - final SynchronousResponse getSpecJob = schedulerClient.createGetSpecJob(imageName); - - if (getSpecJob.isSuccess()) { - LOGGER.info( - "migrateAllDefinitionsToContainSpec - Spec for Source Definition {} was successfully retrieved. Writing to the db...", - sourceDef.getName()); - final StandardSourceDefinition updatedDef = Jsons.clone(sourceDef).withSpec(getSpecJob.getOutput()); - configRepository.writeStandardSourceDefinition(updatedDef); - LOGGER.info( - "migrateAllDefinitionsToContainSpec - Spec for Source Definition {} was successfully written to the db record.", - sourceDef.getName()); - } else { - final LogRead logRead = jobConverter.getLogRead(getSpecJob.getMetadata().getLogPath()); - LOGGER.info( - "migrateAllDefinitionsToContainSpec - Failed to retrieve spec for Source Definition {}. Logs: {}", - sourceDef.getName(), - logRead.toString()); - trackSpecBackfillFailure( - trackingClient, - configRepository, - sourceDef.getDockerRepository(), - sourceDef.getDockerImageTag(), - connectorReposInUse.contains(sourceDef.getDockerRepository()), - configs.getVersion32ForceUpgrade(), - logRead.toString()); - if (configs.getVersion32ForceUpgrade()) { - LOGGER.info( - "migrateAllDefinitionsToContainSpec - Force upgrade set to true. Deleting Source Definition {} and any associated connections/syncs...", - sourceDef.getName()); - configRepository.deleteSourceDefinitionAndAssociations(sourceDef.getSourceDefinitionId()); - } else { - failedBackfillImages.add(imageName); - } - } - } - - final Set seedDestRepos = seed.listConfigs(ConfigSchema.STANDARD_DESTINATION_DEFINITION, StandardDestinationDefinition.class) - .stream() - .map(StandardDestinationDefinition::getDockerRepository) - .collect(Collectors.toSet()); - for (final StandardDestinationDefinition destDef : configRepository.listStandardDestinationDefinitions()) { - // If destination definition already has a spec, there is nothing to backfill - if (destDef.getSpec() != null) { - continue; - } - - final String imageName = destDef.getDockerRepository() + ":" + destDef.getDockerImageTag(); - - // if a source definition is not being used and is not in the seed, don't bother to attempt to fetch - // a spec for it; just delete. - if (!connectorReposInUse.contains(destDef.getDockerRepository()) && !seedDestRepos.contains(destDef.getDockerRepository())) { - LOGGER.info( - "migrateAllDefinitionsToContainSpec - Destination Definition {} does not have a spec, is not in the seed, and is not currently used in a connection. Deleting...", - destDef.getName()); - configRepository.deleteDestinationDefinitionAndAssociations(destDef.getDestinationDefinitionId()); - continue; - } - - LOGGER.info( - "migrateAllDefinitionsToContainSpec - Destination Definition {} does not have a spec. Attempting to retrieve spec...", - destDef.getName()); - final SynchronousResponse getSpecJob = schedulerClient.createGetSpecJob(imageName); - - if (getSpecJob.isSuccess()) { - LOGGER.info( - "migrateAllDefinitionsToContainSpec - Spec for Destination Definition {} was successfully retrieved. Writing to the db...", - destDef.getName()); - final StandardDestinationDefinition updatedDef = Jsons.clone(destDef).withSpec(getSpecJob.getOutput()); - configRepository.writeStandardDestinationDefinition(updatedDef); - LOGGER.info( - "migrateAllDefinitionsToContainSpec - Spec for Destination Definition {} was successfully written to the db record.", - destDef.getName()); - } else { - final LogRead logRead = jobConverter.getLogRead(getSpecJob.getMetadata().getLogPath()); - LOGGER.info( - "migrateAllDefinitionsToContainSpec - Failed to retrieve spec for Destination Definition {}. Logs: {}", - destDef.getName(), - logRead.toString()); - trackSpecBackfillFailure( - trackingClient, - configRepository, - destDef.getDockerRepository(), - destDef.getDockerImageTag(), - connectorReposInUse.contains(destDef.getDockerRepository()), - configs.getVersion32ForceUpgrade(), - logRead.toString()); - if (configs.getVersion32ForceUpgrade()) { - LOGGER.info( - "migrateAllDefinitionsToContainSpec - Force upgrade set to true. Deleting Destination Definition {} and any associated connections/syncs...", - destDef.getName()); - configRepository.deleteDestinationDefinitionAndAssociations(destDef.getDestinationDefinitionId()); - } else { - failedBackfillImages.add(imageName); - } - } - } - - if (failedBackfillImages.size() > 0 && !configs.getVersion32ForceUpgrade()) { - final String attentionBanner = MoreResources.readResource("banner/attention-banner.txt"); - LOGGER.error(attentionBanner); - final String errorMessage = String.format( - "Specs could not be retrieved for the following connector images: %s. Upgrading to version %s " - + "requires specs to be retrieved for all connector definitions, so you must either fix the images or restart the deployment with " - + "the VERSION_0_32_0_FORCE_UPGRADE environment variable set to true, which will cause any connector definitions for which specs " - + "cannot be retrieved to be deleted, as well as their associated connections/syncs.", - failedBackfillImages.toString(), - VERSION_BREAK); - LOGGER.error(errorMessage); - throw new RuntimeException(errorMessage); - } - } - - private static void trackSpecBackfillFailure(final TrackingClient trackingClient, - final ConfigRepository configRepository, - final String dockerRepo, - final String dockerImageTag, - final boolean connectorInUse, - final boolean forceUpgrade, - final String logs) - throws JsonValidationException, IOException { - // There is guaranteed to be at least one workspace, because the getServer() function enforces that - final UUID workspaceId = configRepository.listStandardWorkspaces(true).get(0).getWorkspaceId(); - - final ImmutableMap metadata = ImmutableMap.of( - "docker_image_name", dockerRepo, - "docker_image_tag", dockerImageTag, - "force_upgrade", forceUpgrade, - "connector_in_use", connectorInUse, - "logs", logs); - trackingClient.track(workspaceId, "failed_spec_backfill_major_bump", metadata); - } - -} diff --git a/airbyte-server/src/main/java/io/airbyte/server/RunMigration.java b/airbyte-server/src/main/java/io/airbyte/server/RunMigration.java index 53ba6f2f7521..59e3f25b38d7 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/RunMigration.java +++ b/airbyte-server/src/main/java/io/airbyte/server/RunMigration.java @@ -10,7 +10,6 @@ import io.airbyte.migrate.MigrateConfig; import io.airbyte.migrate.MigrationRunner; import io.airbyte.scheduler.persistence.JobPersistence; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.validation.json.JsonValidationException; import java.io.File; import java.io.IOException; @@ -34,12 +33,11 @@ public class RunMigration implements Runnable, AutoCloseable { public RunMigration(final JobPersistence jobPersistence, final ConfigRepository configRepository, final AirbyteVersion targetVersion, - final ConfigPersistence seedPersistence, - final SpecFetcher specFetcher) { + final ConfigPersistence seedPersistence) { this.targetVersion = targetVersion; this.seedPersistence = seedPersistence; this.configDumpExporter = new ConfigDumpExporter(configRepository, jobPersistence, null); - this.configDumpImporter = new ConfigDumpImporter(configRepository, jobPersistence, null, specFetcher, false); + this.configDumpImporter = new ConfigDumpImporter(configRepository, jobPersistence, null, false); } @Override diff --git a/airbyte-server/src/main/java/io/airbyte/server/ServerApp.java b/airbyte-server/src/main/java/io/airbyte/server/ServerApp.java index ddccd9d36cc9..b86f83acef38 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/ServerApp.java +++ b/airbyte-server/src/main/java/io/airbyte/server/ServerApp.java @@ -8,7 +8,6 @@ import io.airbyte.analytics.Deployment; import io.airbyte.analytics.TrackingClient; import io.airbyte.analytics.TrackingClientSingleton; -import io.airbyte.commons.lang.Exceptions; import io.airbyte.commons.resources.MoreResources; import io.airbyte.commons.version.AirbyteVersion; import io.airbyte.config.Configs; @@ -28,18 +27,14 @@ import io.airbyte.db.instance.configs.ConfigsDatabaseMigrator; import io.airbyte.db.instance.jobs.JobsDatabaseInstance; import io.airbyte.db.instance.jobs.JobsDatabaseMigrator; -import io.airbyte.scheduler.client.BucketSpecCacheSchedulerClient; import io.airbyte.scheduler.client.DefaultSchedulerJobClient; import io.airbyte.scheduler.client.DefaultSynchronousSchedulerClient; import io.airbyte.scheduler.client.SchedulerJobClient; -import io.airbyte.scheduler.client.SpecCachingSynchronousSchedulerClient; -import io.airbyte.scheduler.client.SynchronousSchedulerClient; import io.airbyte.scheduler.persistence.DefaultJobCreator; import io.airbyte.scheduler.persistence.DefaultJobPersistence; import io.airbyte.scheduler.persistence.JobPersistence; import io.airbyte.scheduler.persistence.job_factory.OAuthConfigSupplier; import io.airbyte.scheduler.persistence.job_tracker.JobTracker; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.server.errors.InvalidInputExceptionMapper; import io.airbyte.server.errors.InvalidJsonExceptionMapper; import io.airbyte.server.errors.InvalidJsonInputExceptionMapper; @@ -217,9 +212,6 @@ public static ServerRunnable getServer(final ServerFactory apiFactory, final Con new DefaultSchedulerJobClient(jobPersistence, new DefaultJobCreator(jobPersistence, configRepository)); final DefaultSynchronousSchedulerClient syncSchedulerClient = new DefaultSynchronousSchedulerClient(temporalClient, jobTracker, oAuthConfigSupplier); - final SynchronousSchedulerClient bucketSpecCacheSchedulerClient = - new BucketSpecCacheSchedulerClient(syncSchedulerClient, configs.getSpecCacheBucket()); - final SpecCachingSynchronousSchedulerClient cachingSchedulerClient = new SpecCachingSynchronousSchedulerClient(bucketSpecCacheSchedulerClient); final HttpClient httpClient = HttpClient.newBuilder().version(HttpClient.Version.HTTP_1_1).build(); // version in the database when the server main method is called. may be empty if this is the first @@ -249,20 +241,9 @@ public static ServerRunnable getServer(final ServerFactory apiFactory, final Con configPersistence.loadData(seed); LOGGER.info("Loaded seed data..."); - // todo (lmossman) - this will only exist temporarily to ensure all definitions contain specs. It - // will be removed after the faux major version bump - ConnectorDefinitionSpecBackfiller.migrateAllDefinitionsToContainSpec( - configRepository, - configPersistence, - seed, - cachingSchedulerClient, - trackingClient, - configs); - LOGGER.info("Migrated all definitions to contain specs..."); - return apiFactory.create( schedulerJobClient, - cachingSchedulerClient, + syncSchedulerClient, temporalService, configRepository, jobPersistence, @@ -306,21 +287,16 @@ private static AirbyteVersion runFileMigration(final AirbyteVersion airbyteVersi final AirbyteVersion initialAirbyteDatabaseVersion, final ConfigRepository configRepository, final ConfigPersistence seed, - final SpecFetcher specFetcher, final JobPersistence jobPersistence, final Configs configs) throws IOException { - // required before migration - // TODO: remove this specFetcherFn logic once file migrations are deprecated - configRepository.setSpecFetcher(dockerImage -> Exceptions.toRuntime(() -> specFetcher.getSpec(dockerImage))); - // version in the database after migration is run. AirbyteVersion airbyteDatabaseVersion = null; if (initialAirbyteDatabaseVersion != null && isDatabaseVersionBehindAppVersion(airbyteVersion, initialAirbyteDatabaseVersion)) { final boolean isKubernetes = configs.getWorkerEnvironment() == WorkerEnvironment.KUBERNETES; final boolean versionSupportsAutoMigrate = initialAirbyteDatabaseVersion.greaterThanOrEqualTo(KUBE_SUPPORT_FOR_AUTOMATIC_MIGRATION); if (!isKubernetes || versionSupportsAutoMigrate) { - runAutomaticMigration(configRepository, jobPersistence, seed, specFetcher, airbyteVersion, initialAirbyteDatabaseVersion); + runAutomaticMigration(configRepository, jobPersistence, seed, airbyteVersion, initialAirbyteDatabaseVersion); // After migration, upgrade the DB version airbyteDatabaseVersion = jobPersistence.getVersion().map(AirbyteVersion::new).orElseThrow(); } else { @@ -342,7 +318,6 @@ public static void main(final String[] args) throws Exception { private static void runAutomaticMigration(final ConfigRepository configRepository, final JobPersistence jobPersistence, final ConfigPersistence seed, - final SpecFetcher specFetcher, final AirbyteVersion airbyteVersion, final AirbyteVersion airbyteDatabaseVersion) { LOGGER.info("Running Automatic Migration from version : " + airbyteDatabaseVersion.serialize() + " to version : " + airbyteVersion.serialize()); @@ -350,8 +325,7 @@ private static void runAutomaticMigration(final ConfigRepository configRepositor jobPersistence, configRepository, airbyteVersion, - seed, - specFetcher)) { + seed)) { runMigration.run(); } catch (final Exception e) { LOGGER.error("Automatic Migration failed ", e); diff --git a/airbyte-server/src/main/java/io/airbyte/server/ServerFactory.java b/airbyte-server/src/main/java/io/airbyte/server/ServerFactory.java index 1bf5528b9809..c520ef8e1176 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/ServerFactory.java +++ b/airbyte-server/src/main/java/io/airbyte/server/ServerFactory.java @@ -13,7 +13,7 @@ import io.airbyte.config.persistence.ConfigRepository; import io.airbyte.db.Database; import io.airbyte.scheduler.client.SchedulerJobClient; -import io.airbyte.scheduler.client.SpecCachingSynchronousSchedulerClient; +import io.airbyte.scheduler.client.SynchronousSchedulerClient; import io.airbyte.scheduler.persistence.JobPersistence; import io.airbyte.server.apis.ConfigurationApi; import io.temporal.serviceclient.WorkflowServiceStubs; @@ -26,7 +26,7 @@ public interface ServerFactory { ServerRunnable create(SchedulerJobClient schedulerJobClient, - SpecCachingSynchronousSchedulerClient cachingSchedulerClient, + SynchronousSchedulerClient cachingSchedulerClient, WorkflowServiceStubs temporalService, ConfigRepository configRepository, JobPersistence jobPersistence, @@ -45,7 +45,7 @@ class Api implements ServerFactory { @Override public ServerRunnable create(final SchedulerJobClient schedulerJobClient, - final SpecCachingSynchronousSchedulerClient cachingSchedulerClient, + final SynchronousSchedulerClient synchronousSchedulerClient, final WorkflowServiceStubs temporalService, final ConfigRepository configRepository, final JobPersistence jobPersistence, @@ -66,7 +66,7 @@ public ServerRunnable create(final SchedulerJobClient schedulerJobClient, jobPersistence, seed, schedulerJobClient, - cachingSchedulerClient, + synchronousSchedulerClient, new FileTtlManager(10, TimeUnit.MINUTES, 10), MDC.getCopyOfContextMap(), configsDatabase, diff --git a/airbyte-server/src/main/java/io/airbyte/server/apis/ConfigurationApi.java b/airbyte-server/src/main/java/io/airbyte/server/apis/ConfigurationApi.java index 056fbced6c72..d725a7ea56de 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/apis/ConfigurationApi.java +++ b/airbyte-server/src/main/java/io/airbyte/server/apis/ConfigurationApi.java @@ -89,13 +89,12 @@ import io.airbyte.config.persistence.ConfigPersistence; import io.airbyte.config.persistence.ConfigRepository; import io.airbyte.db.Database; -import io.airbyte.scheduler.client.CachingSynchronousSchedulerClient; import io.airbyte.scheduler.client.SchedulerJobClient; +import io.airbyte.scheduler.client.SynchronousSchedulerClient; import io.airbyte.scheduler.persistence.JobNotifier; import io.airbyte.scheduler.persistence.JobPersistence; import io.airbyte.scheduler.persistence.WorkspaceHelper; import io.airbyte.scheduler.persistence.job_factory.OAuthConfigSupplier; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.server.errors.BadObjectSchemaKnownException; import io.airbyte.server.errors.IdNotFoundKnownException; import io.airbyte.server.handlers.ArchiveHandler; @@ -151,7 +150,7 @@ public ConfigurationApi(final ConfigRepository configRepository, final JobPersistence jobPersistence, final ConfigPersistence seed, final SchedulerJobClient schedulerJobClient, - final CachingSynchronousSchedulerClient synchronousSchedulerClient, + final SynchronousSchedulerClient synchronousSchedulerClient, final FileTtlManager archiveTtlManager, final WorkflowServiceStubs temporalService, final Database configsDatabase, @@ -167,7 +166,6 @@ public ConfigurationApi(final ConfigRepository configRepository, this.logConfigs = logConfigs; this.workspaceRoot = workspaceRoot; - final SpecFetcher specFetcher = new SpecFetcher(synchronousSchedulerClient); final JsonSchemaValidator schemaValidator = new JsonSchemaValidator(); final JobNotifier jobNotifier = new JobNotifier( webappUrl, @@ -187,11 +185,11 @@ public ConfigurationApi(final ConfigRepository configRepository, connectionsHandler = new ConnectionsHandler(configRepository, workspaceHelper, trackingClient); operationsHandler = new OperationsHandler(configRepository); destinationDefinitionsHandler = new DestinationDefinitionsHandler(configRepository, synchronousSchedulerClient); - destinationHandler = new DestinationHandler(configRepository, schemaValidator, specFetcher, connectionsHandler); - sourceHandler = new SourceHandler(configRepository, schemaValidator, specFetcher, connectionsHandler); + destinationHandler = new DestinationHandler(configRepository, schemaValidator, connectionsHandler); + sourceHandler = new SourceHandler(configRepository, schemaValidator, connectionsHandler); workspacesHandler = new WorkspacesHandler(configRepository, connectionsHandler, destinationHandler, sourceHandler); jobHistoryHandler = new JobHistoryHandler(jobPersistence, workerEnvironment, logConfigs); - oAuthHandler = new OAuthHandler(configRepository, httpClient, trackingClient, specFetcher); + oAuthHandler = new OAuthHandler(configRepository, httpClient, trackingClient); webBackendConnectionsHandler = new WebBackendConnectionsHandler( connectionsHandler, sourceHandler, @@ -207,7 +205,6 @@ public ConfigurationApi(final ConfigRepository configRepository, seed, workspaceHelper, archiveTtlManager, - specFetcher, true); logsHandler = new LogsHandler(); openApiConfigHandler = new OpenApiConfigHandler(); diff --git a/airbyte-server/src/main/java/io/airbyte/server/converters/ConfigurationUpdate.java b/airbyte-server/src/main/java/io/airbyte/server/converters/ConfigurationUpdate.java index 1fecb41fb577..99861c0d9529 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/converters/ConfigurationUpdate.java +++ b/airbyte-server/src/main/java/io/airbyte/server/converters/ConfigurationUpdate.java @@ -21,16 +21,14 @@ public class ConfigurationUpdate { private final ConfigRepository configRepository; - private final SpecFetcher specFetcher; private final JsonSecretsProcessor secretsProcessor; - public ConfigurationUpdate(final ConfigRepository configRepository, final SpecFetcher specFetcher) { - this(configRepository, specFetcher, new JsonSecretsProcessor()); + public ConfigurationUpdate(final ConfigRepository configRepository) { + this(configRepository, new JsonSecretsProcessor()); } - public ConfigurationUpdate(final ConfigRepository configRepository, final SpecFetcher specFetcher, final JsonSecretsProcessor secretsProcessor) { + public ConfigurationUpdate(final ConfigRepository configRepository, final JsonSecretsProcessor secretsProcessor) { this.configRepository = configRepository; - this.specFetcher = specFetcher; this.secretsProcessor = secretsProcessor; } @@ -41,7 +39,7 @@ public SourceConnection source(final UUID sourceId, final String sourceName, fin persistedSource.setName(sourceName); // get spec final StandardSourceDefinition sourceDefinition = configRepository.getStandardSourceDefinition(persistedSource.getSourceDefinitionId()); - final ConnectorSpecification spec = specFetcher.getSpec(sourceDefinition); + final ConnectorSpecification spec = sourceDefinition.getSpec(); // copy any necessary secrets from the current source to the incoming updated source final JsonNode updatedConfiguration = secretsProcessor.copySecrets( persistedSource.getConfiguration(), @@ -59,7 +57,7 @@ public DestinationConnection destination(final UUID destinationId, final String // get spec final StandardDestinationDefinition destinationDefinition = configRepository .getStandardDestinationDefinition(persistedDestination.getDestinationDefinitionId()); - final ConnectorSpecification spec = specFetcher.getSpec(destinationDefinition); + final ConnectorSpecification spec = destinationDefinition.getSpec(); // copy any necessary secrets from the current destination to the incoming updated destination final JsonNode updatedConfiguration = secretsProcessor.copySecrets( persistedDestination.getConfiguration(), diff --git a/airbyte-server/src/main/java/io/airbyte/server/converters/SpecFetcher.java b/airbyte-server/src/main/java/io/airbyte/server/converters/SpecFetcher.java index bd9a10d40183..f60e75ff17bb 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/converters/SpecFetcher.java +++ b/airbyte-server/src/main/java/io/airbyte/server/converters/SpecFetcher.java @@ -5,77 +5,11 @@ package io.airbyte.server.converters; import com.google.common.base.Preconditions; -import io.airbyte.commons.docker.DockerUtils; -import io.airbyte.config.JobConfig.ConfigType; -import io.airbyte.config.StandardDestinationDefinition; -import io.airbyte.config.StandardSourceDefinition; import io.airbyte.protocol.models.ConnectorSpecification; -import io.airbyte.scheduler.client.SynchronousJobMetadata; import io.airbyte.scheduler.client.SynchronousResponse; -import io.airbyte.scheduler.client.SynchronousSchedulerClient; -import java.io.IOException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class SpecFetcher { - private static final Logger LOGGER = LoggerFactory.getLogger(SpecFetcher.class); - - private final SynchronousSchedulerClient schedulerJobClient; - - public SpecFetcher(final SynchronousSchedulerClient schedulerJobClient) { - this.schedulerJobClient = schedulerJobClient; - } - - // TODO: remove this once file migrations are deprecated, as that is the only time this function is - // used - @Deprecated - public ConnectorSpecification getSpec(final String dockerImage) throws IOException { - return getSpecFromJob(schedulerJobClient.createGetSpecJob(dockerImage)); - } - - public ConnectorSpecification getSpec(final StandardSourceDefinition sourceDefinition) throws IOException { - return getSpecFromJob(getSpecJobResponse(sourceDefinition)); - } - - public ConnectorSpecification getSpec(final StandardDestinationDefinition destinationDefinition) throws IOException { - return getSpecFromJob(getSpecJobResponse(destinationDefinition)); - } - - // TODO: remove this method once the spec is a required field on the StandardSourceDefinition struct - public SynchronousResponse getSpecJobResponse(final StandardSourceDefinition sourceDefinition) throws IOException { - LOGGER.debug("Spec Fetcher: Getting spec for Source Definition."); - final ConnectorSpecification spec = sourceDefinition.getSpec(); - - if (spec != null) { - LOGGER.debug("Spec Fetcher: Spec found in Source Definition."); - return new SynchronousResponse<>(spec, SynchronousJobMetadata.mock(ConfigType.GET_SPEC)); - } - - LOGGER.debug("Spec Fetcher: Spec not found in Source Definition, fetching with scheduler job instead."); - final String dockerImageName = DockerUtils.getTaggedImageName(sourceDefinition.getDockerRepository(), sourceDefinition.getDockerImageTag()); - return schedulerJobClient.createGetSpecJob(dockerImageName); - } - - // TODO: remove this method once the spec is a required field on the StandardDestinationDefinition - // struct - public SynchronousResponse getSpecJobResponse(final StandardDestinationDefinition destinationDefinition) - throws IOException { - LOGGER.debug("Spec Fetcher: Getting spec for Destination Definition."); - final ConnectorSpecification spec = destinationDefinition.getSpec(); - - if (spec != null) { - LOGGER.debug("Spec Fetcher: Spec found in Destination Definition."); - return new SynchronousResponse<>(spec, SynchronousJobMetadata.mock(ConfigType.GET_SPEC)); - } - - LOGGER.debug("Spec Fetcher: Spec not found in Destination Definition, fetching with scheduler job instead."); - final String dockerImageName = DockerUtils.getTaggedImageName( - destinationDefinition.getDockerRepository(), - destinationDefinition.getDockerImageTag()); - return schedulerJobClient.createGetSpecJob(dockerImageName); - } - public static ConnectorSpecification getSpecFromJob(final SynchronousResponse response) { Preconditions.checkState(response.isSuccess(), "Get Spec job failed."); Preconditions.checkNotNull(response.getOutput(), "Get Spec job return null spec"); diff --git a/airbyte-server/src/main/java/io/airbyte/server/handlers/ArchiveHandler.java b/airbyte-server/src/main/java/io/airbyte/server/handlers/ArchiveHandler.java index a5ad765ef62d..62b3c4a32725 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/handlers/ArchiveHandler.java +++ b/airbyte-server/src/main/java/io/airbyte/server/handlers/ArchiveHandler.java @@ -18,7 +18,6 @@ import io.airbyte.scheduler.persistence.WorkspaceHelper; import io.airbyte.server.ConfigDumpExporter; import io.airbyte.server.ConfigDumpImporter; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.server.errors.InternalServerKnownException; import io.airbyte.validation.json.JsonValidationException; import java.io.File; @@ -43,13 +42,12 @@ public ArchiveHandler(final AirbyteVersion version, final ConfigPersistence seed, final WorkspaceHelper workspaceHelper, final FileTtlManager fileTtlManager, - final SpecFetcher specFetcher, final boolean importDefinitions) { this( version, fileTtlManager, new ConfigDumpExporter(configRepository, jobPersistence, workspaceHelper), - new ConfigDumpImporter(configRepository, jobPersistence, workspaceHelper, specFetcher, importDefinitions), + new ConfigDumpImporter(configRepository, jobPersistence, workspaceHelper, importDefinitions), seed); } diff --git a/airbyte-server/src/main/java/io/airbyte/server/handlers/DestinationDefinitionsHandler.java b/airbyte-server/src/main/java/io/airbyte/server/handlers/DestinationDefinitionsHandler.java index 459e4dae396c..f9713f18d7ee 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/handlers/DestinationDefinitionsHandler.java +++ b/airbyte-server/src/main/java/io/airbyte/server/handlers/DestinationDefinitionsHandler.java @@ -16,8 +16,8 @@ import io.airbyte.config.persistence.ConfigNotFoundException; import io.airbyte.config.persistence.ConfigRepository; import io.airbyte.protocol.models.ConnectorSpecification; -import io.airbyte.scheduler.client.CachingSynchronousSchedulerClient; import io.airbyte.scheduler.client.SynchronousResponse; +import io.airbyte.scheduler.client.SynchronousSchedulerClient; import io.airbyte.server.converters.SpecFetcher; import io.airbyte.server.errors.InternalServerKnownException; import io.airbyte.server.services.AirbyteGithubStore; @@ -38,18 +38,18 @@ public class DestinationDefinitionsHandler { private final ConfigRepository configRepository; private final Supplier uuidSupplier; - private final CachingSynchronousSchedulerClient schedulerSynchronousClient; + private final SynchronousSchedulerClient schedulerSynchronousClient; private final AirbyteGithubStore githubStore; public DestinationDefinitionsHandler(final ConfigRepository configRepository, - final CachingSynchronousSchedulerClient schedulerSynchronousClient) { + final SynchronousSchedulerClient schedulerSynchronousClient) { this(configRepository, UUID::randomUUID, schedulerSynchronousClient, AirbyteGithubStore.production()); } @VisibleForTesting public DestinationDefinitionsHandler(final ConfigRepository configRepository, final Supplier uuidSupplier, - final CachingSynchronousSchedulerClient schedulerSynchronousClient, + final SynchronousSchedulerClient schedulerSynchronousClient, final AirbyteGithubStore githubStore) { this.configRepository = configRepository; this.uuidSupplier = uuidSupplier; @@ -128,9 +128,7 @@ public DestinationDefinitionRead updateDestinationDefinition(final DestinationDe .getStandardDestinationDefinition(destinationDefinitionUpdate.getDestinationDefinitionId()); final boolean imageTagHasChanged = !currentDestination.getDockerImageTag().equals(destinationDefinitionUpdate.getDockerImageTag()); - // TODO (lmossman): remove null spec condition when the spec field becomes required on the - // definition struct - final ConnectorSpecification spec = (imageTagHasChanged || currentDestination.getSpec() == null) + final ConnectorSpecification spec = imageTagHasChanged ? getSpecForImage(currentDestination.getDockerRepository(), destinationDefinitionUpdate.getDockerImageTag()) : currentDestination.getSpec(); @@ -144,8 +142,6 @@ public DestinationDefinitionRead updateDestinationDefinition(final DestinationDe .withSpec(spec); configRepository.writeStandardDestinationDefinition(newDestination); - // we want to re-fetch the spec for updated definitions. - schedulerSynchronousClient.resetCache(); return buildDestinationDefinitionRead(newDestination); } diff --git a/airbyte-server/src/main/java/io/airbyte/server/handlers/DestinationHandler.java b/airbyte-server/src/main/java/io/airbyte/server/handlers/DestinationHandler.java index 5c85065240a4..87322700528c 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/handlers/DestinationHandler.java +++ b/airbyte-server/src/main/java/io/airbyte/server/handlers/DestinationHandler.java @@ -24,7 +24,6 @@ import io.airbyte.config.persistence.split_secrets.JsonSecretsProcessor; import io.airbyte.protocol.models.ConnectorSpecification; import io.airbyte.server.converters.ConfigurationUpdate; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.validation.json.JsonSchemaValidator; import io.airbyte.validation.json.JsonValidationException; import java.io.IOException; @@ -39,7 +38,6 @@ public class DestinationHandler { private static final Logger LOGGER = LoggerFactory.getLogger(DestinationHandler.class); private final ConnectionsHandler connectionsHandler; - private final SpecFetcher specFetcher; private final Supplier uuidGenerator; private final ConfigRepository configRepository; private final JsonSchemaValidator validator; @@ -49,14 +47,12 @@ public class DestinationHandler { @VisibleForTesting DestinationHandler(final ConfigRepository configRepository, final JsonSchemaValidator integrationSchemaValidation, - final SpecFetcher specFetcher, final ConnectionsHandler connectionsHandler, final Supplier uuidGenerator, final JsonSecretsProcessor secretsProcessor, final ConfigurationUpdate configurationUpdate) { this.configRepository = configRepository; this.validator = integrationSchemaValidation; - this.specFetcher = specFetcher; this.connectionsHandler = connectionsHandler; this.uuidGenerator = uuidGenerator; this.configurationUpdate = configurationUpdate; @@ -65,16 +61,14 @@ public class DestinationHandler { public DestinationHandler(final ConfigRepository configRepository, final JsonSchemaValidator integrationSchemaValidation, - final SpecFetcher specFetcher, final ConnectionsHandler connectionsHandler) { this( configRepository, integrationSchemaValidation, - specFetcher, connectionsHandler, UUID::randomUUID, new JsonSecretsProcessor(), - new ConfigurationUpdate(configRepository, specFetcher)); + new ConfigurationUpdate(configRepository)); } public DestinationRead createDestination(final DestinationCreate destinationCreate) @@ -207,12 +201,7 @@ private void validateDestination(final ConnectorSpecification spec, final JsonNo public ConnectorSpecification getSpec(final UUID destinationDefinitionId) throws JsonValidationException, IOException, ConfigNotFoundException { - return getSpec(specFetcher, configRepository.getStandardDestinationDefinition(destinationDefinitionId)); - } - - public static ConnectorSpecification getSpec(final SpecFetcher specFetcher, final StandardDestinationDefinition destinationDef) - throws JsonValidationException, IOException, ConfigNotFoundException { - return specFetcher.getSpec(destinationDef); + return configRepository.getStandardDestinationDefinition(destinationDefinitionId).getSpec(); } private void persistDestinationConnection(final String name, diff --git a/airbyte-server/src/main/java/io/airbyte/server/handlers/OAuthHandler.java b/airbyte-server/src/main/java/io/airbyte/server/handlers/OAuthHandler.java index 83460cd48461..27f938287cfe 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/handlers/OAuthHandler.java +++ b/airbyte-server/src/main/java/io/airbyte/server/handlers/OAuthHandler.java @@ -25,7 +25,6 @@ import io.airbyte.protocol.models.ConnectorSpecification; import io.airbyte.scheduler.persistence.job_factory.OAuthConfigSupplier; import io.airbyte.scheduler.persistence.job_tracker.TrackingMetadata; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.validation.json.JsonValidationException; import java.io.IOException; import java.net.http.HttpClient; @@ -41,16 +40,13 @@ public class OAuthHandler { private final ConfigRepository configRepository; private final OAuthImplementationFactory oAuthImplementationFactory; private final TrackingClient trackingClient; - private final SpecFetcher specFetcher; public OAuthHandler(final ConfigRepository configRepository, final HttpClient httpClient, - final TrackingClient trackingClient, - final SpecFetcher specFetcher) { + final TrackingClient trackingClient) { this.configRepository = configRepository; this.oAuthImplementationFactory = new OAuthImplementationFactory(configRepository, httpClient); this.trackingClient = trackingClient; - this.specFetcher = specFetcher; } public OAuthConsentRead getSourceOAuthConsent(final SourceOauthConsentRequest sourceDefinitionIdRequestBody) @@ -58,7 +54,7 @@ public OAuthConsentRead getSourceOAuthConsent(final SourceOauthConsentRequest so final StandardSourceDefinition sourceDefinition = configRepository.getStandardSourceDefinition(sourceDefinitionIdRequestBody.getSourceDefinitionId()); final OAuthFlowImplementation oAuthFlowImplementation = oAuthImplementationFactory.create(sourceDefinition); - final ConnectorSpecification spec = specFetcher.getSpec(sourceDefinition); + final ConnectorSpecification spec = sourceDefinition.getSpec(); final ImmutableMap metadata = generateSourceMetadata(sourceDefinitionIdRequestBody.getSourceDefinitionId()); final OAuthConsentRead result; if (OAuthConfigSupplier.hasOAuthConfigSpecification(spec)) { @@ -87,7 +83,7 @@ public OAuthConsentRead getDestinationOAuthConsent(final DestinationOauthConsent final StandardDestinationDefinition destinationDefinition = configRepository.getStandardDestinationDefinition(destinationDefinitionIdRequestBody.getDestinationDefinitionId()); final OAuthFlowImplementation oAuthFlowImplementation = oAuthImplementationFactory.create(destinationDefinition); - final ConnectorSpecification spec = specFetcher.getSpec(destinationDefinition); + final ConnectorSpecification spec = destinationDefinition.getSpec(); final ImmutableMap metadata = generateDestinationMetadata(destinationDefinitionIdRequestBody.getDestinationDefinitionId()); final OAuthConsentRead result; if (OAuthConfigSupplier.hasOAuthConfigSpecification(spec)) { @@ -115,7 +111,7 @@ public Map completeSourceOAuth(final CompleteSourceOauthRequest throws JsonValidationException, ConfigNotFoundException, IOException { final StandardSourceDefinition sourceDefinition = configRepository.getStandardSourceDefinition(oauthSourceRequestBody.getSourceDefinitionId()); final OAuthFlowImplementation oAuthFlowImplementation = oAuthImplementationFactory.create(sourceDefinition); - final ConnectorSpecification spec = specFetcher.getSpec(sourceDefinition); + final ConnectorSpecification spec = sourceDefinition.getSpec(); final ImmutableMap metadata = generateSourceMetadata(oauthSourceRequestBody.getSourceDefinitionId()); final Map result; if (OAuthConfigSupplier.hasOAuthConfigSpecification(spec)) { @@ -147,7 +143,7 @@ public Map completeDestinationOAuth(final CompleteDestinationOAu final StandardDestinationDefinition destinationDefinition = configRepository.getStandardDestinationDefinition(oauthDestinationRequestBody.getDestinationDefinitionId()); final OAuthFlowImplementation oAuthFlowImplementation = oAuthImplementationFactory.create(destinationDefinition); - final ConnectorSpecification spec = specFetcher.getSpec(destinationDefinition); + final ConnectorSpecification spec = destinationDefinition.getSpec(); final ImmutableMap metadata = generateDestinationMetadata(oauthDestinationRequestBody.getDestinationDefinitionId()); final Map result; if (OAuthConfigSupplier.hasOAuthConfigSpecification(spec)) { diff --git a/airbyte-server/src/main/java/io/airbyte/server/handlers/SchedulerHandler.java b/airbyte-server/src/main/java/io/airbyte/server/handlers/SchedulerHandler.java index 3d972fb2fe28..77b56dac6548 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/handlers/SchedulerHandler.java +++ b/airbyte-server/src/main/java/io/airbyte/server/handlers/SchedulerHandler.java @@ -31,6 +31,7 @@ import io.airbyte.commons.enums.Enums; import io.airbyte.config.Configs.WorkerEnvironment; 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.StandardDestinationDefinition; @@ -44,6 +45,7 @@ import io.airbyte.protocol.models.AirbyteCatalog; import io.airbyte.protocol.models.ConnectorSpecification; import io.airbyte.scheduler.client.SchedulerJobClient; +import io.airbyte.scheduler.client.SynchronousJobMetadata; import io.airbyte.scheduler.client.SynchronousResponse; import io.airbyte.scheduler.client.SynchronousSchedulerClient; import io.airbyte.scheduler.models.Job; @@ -54,7 +56,6 @@ import io.airbyte.server.converters.ConfigurationUpdate; import io.airbyte.server.converters.JobConverter; import io.airbyte.server.converters.OauthModelConverter; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.validation.json.JsonSchemaValidator; import io.airbyte.validation.json.JsonValidationException; import io.airbyte.workers.temporal.TemporalUtils; @@ -75,7 +76,6 @@ public class SchedulerHandler { private final ConfigRepository configRepository; private final SchedulerJobClient schedulerJobClient; private final SynchronousSchedulerClient synchronousSchedulerClient; - private final SpecFetcher specFetcher; private final ConfigurationUpdate configurationUpdate; private final JsonSchemaValidator jsonSchemaValidator; private final JobPersistence jobPersistence; @@ -99,9 +99,8 @@ public SchedulerHandler(final ConfigRepository configRepository, configRepository, schedulerJobClient, synchronousSchedulerClient, - new ConfigurationUpdate(configRepository, new SpecFetcher(synchronousSchedulerClient)), + new ConfigurationUpdate(configRepository), new JsonSchemaValidator(), - new SpecFetcher(synchronousSchedulerClient), jobPersistence, jobNotifier, temporalService, @@ -116,7 +115,6 @@ public SchedulerHandler(final ConfigRepository configRepository, final SynchronousSchedulerClient synchronousSchedulerClient, final ConfigurationUpdate configurationUpdate, final JsonSchemaValidator jsonSchemaValidator, - final SpecFetcher specFetcher, final JobPersistence jobPersistence, final JobNotifier jobNotifier, final WorkflowServiceStubs temporalService, @@ -128,7 +126,6 @@ public SchedulerHandler(final ConfigRepository configRepository, this.synchronousSchedulerClient = synchronousSchedulerClient; this.configurationUpdate = configurationUpdate; this.jsonSchemaValidator = jsonSchemaValidator; - this.specFetcher = specFetcher; this.jobPersistence = jobPersistence; this.jobNotifier = jobNotifier; this.temporalService = temporalService; @@ -152,7 +149,7 @@ public CheckConnectionRead checkSourceConnectionFromSourceCreate(final SourceCor final StandardSourceDefinition sourceDef = configRepository.getStandardSourceDefinition(sourceConfig.getSourceDefinitionId()); final var partialConfig = configRepository.statefulSplitEphemeralSecrets( sourceConfig.getConnectionConfiguration(), - specFetcher.getSpec(sourceDef)); + sourceDef.getSpec()); // todo (cgardens) - narrow the struct passed to the client. we are not setting fields that are // technically declared as required. @@ -192,7 +189,7 @@ public CheckConnectionRead checkDestinationConnectionFromDestinationCreate(final final StandardDestinationDefinition destDef = configRepository.getStandardDestinationDefinition(destinationConfig.getDestinationDefinitionId()); final var partialConfig = configRepository.statefulSplitEphemeralSecrets( destinationConfig.getConnectionConfiguration(), - specFetcher.getSpec(destDef)); + destDef.getSpec()); // todo (cgardens) - narrow the struct passed to the client. we are not setting fields that are // technically declared as required. @@ -256,10 +253,9 @@ public SourceDefinitionSpecificationRead getSourceDefinitionSpecification(final throws ConfigNotFoundException, IOException, JsonValidationException { final UUID sourceDefinitionId = sourceDefinitionIdRequestBody.getSourceDefinitionId(); final StandardSourceDefinition source = configRepository.getStandardSourceDefinition(sourceDefinitionId); - final SynchronousResponse response = specFetcher.getSpecJobResponse(source); - final ConnectorSpecification spec = response.getOutput(); + final ConnectorSpecification spec = source.getSpec(); final SourceDefinitionSpecificationRead specRead = new SourceDefinitionSpecificationRead() - .jobInfo(jobConverter.getSynchronousJobRead(response)) + .jobInfo(jobConverter.getSynchronousJobRead(SynchronousJobMetadata.mock(ConfigType.GET_SPEC))) .connectionSpecification(spec.getConnectionSpecification()) .documentationUrl(spec.getDocumentationUrl().toString()) .sourceDefinitionId(sourceDefinitionId); @@ -278,11 +274,10 @@ public DestinationDefinitionSpecificationRead getDestinationSpecification( throws ConfigNotFoundException, IOException, JsonValidationException { final UUID destinationDefinitionId = destinationDefinitionIdRequestBody.getDestinationDefinitionId(); final StandardDestinationDefinition destination = configRepository.getStandardDestinationDefinition(destinationDefinitionId); - final SynchronousResponse response = specFetcher.getSpecJobResponse(destination); - final ConnectorSpecification spec = response.getOutput(); + final ConnectorSpecification spec = destination.getSpec(); final DestinationDefinitionSpecificationRead specRead = new DestinationDefinitionSpecificationRead() - .jobInfo(jobConverter.getSynchronousJobRead(response)) + .jobInfo(jobConverter.getSynchronousJobRead(SynchronousJobMetadata.mock(ConfigType.GET_SPEC))) .supportedDestinationSyncModes(Enums.convertListTo(spec.getSupportedDestinationSyncModes(), DestinationSyncMode.class)) .connectionSpecification(spec.getConnectionSpecification()) .documentationUrl(spec.getDocumentationUrl().toString()) @@ -426,13 +421,13 @@ private CheckConnectionRead reportConnectionStatus(final SynchronousResponse uuidSupplier; private final AirbyteGithubStore githubStore; - private final CachingSynchronousSchedulerClient schedulerSynchronousClient; + private final SynchronousSchedulerClient schedulerSynchronousClient; public SourceDefinitionsHandler( final ConfigRepository configRepository, - final CachingSynchronousSchedulerClient schedulerSynchronousClient) { + final SynchronousSchedulerClient schedulerSynchronousClient) { this(configRepository, UUID::randomUUID, schedulerSynchronousClient, AirbyteGithubStore.production()); } public SourceDefinitionsHandler( final ConfigRepository configRepository, final Supplier uuidSupplier, - final CachingSynchronousSchedulerClient schedulerSynchronousClient, + final SynchronousSchedulerClient schedulerSynchronousClient, final AirbyteGithubStore githubStore) { this.configRepository = configRepository; this.uuidSupplier = uuidSupplier; @@ -122,9 +122,7 @@ public SourceDefinitionRead updateSourceDefinition(final SourceDefinitionUpdate configRepository.getStandardSourceDefinition(sourceDefinitionUpdate.getSourceDefinitionId()); final boolean imageTagHasChanged = !currentSourceDefinition.getDockerImageTag().equals(sourceDefinitionUpdate.getDockerImageTag()); - // TODO (lmossman): remove null spec condition when the spec field becomes required on the - // definition struct - final ConnectorSpecification spec = (imageTagHasChanged || currentSourceDefinition.getSpec() == null) + final ConnectorSpecification spec = imageTagHasChanged ? getSpecForImage(currentSourceDefinition.getDockerRepository(), sourceDefinitionUpdate.getDockerImageTag()) : currentSourceDefinition.getSpec(); @@ -138,8 +136,6 @@ public SourceDefinitionRead updateSourceDefinition(final SourceDefinitionUpdate .withSpec(spec); configRepository.writeStandardSourceDefinition(newSource); - // we want to re-fetch the spec for updated definitions. - schedulerSynchronousClient.resetCache(); return buildSourceDefinitionRead(newSource); } diff --git a/airbyte-server/src/main/java/io/airbyte/server/handlers/SourceHandler.java b/airbyte-server/src/main/java/io/airbyte/server/handlers/SourceHandler.java index e41259ce5dc3..d49f9780095a 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/handlers/SourceHandler.java +++ b/airbyte-server/src/main/java/io/airbyte/server/handlers/SourceHandler.java @@ -22,7 +22,6 @@ import io.airbyte.config.persistence.split_secrets.JsonSecretsProcessor; import io.airbyte.protocol.models.ConnectorSpecification; import io.airbyte.server.converters.ConfigurationUpdate; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.validation.json.JsonSchemaValidator; import io.airbyte.validation.json.JsonValidationException; import java.io.IOException; @@ -35,21 +34,18 @@ public class SourceHandler { private final Supplier uuidGenerator; private final ConfigRepository configRepository; private final JsonSchemaValidator validator; - private final SpecFetcher specFetcher; private final ConnectionsHandler connectionsHandler; private final ConfigurationUpdate configurationUpdate; private final JsonSecretsProcessor secretsProcessor; SourceHandler(final ConfigRepository configRepository, final JsonSchemaValidator integrationSchemaValidation, - final SpecFetcher specFetcher, final ConnectionsHandler connectionsHandler, final Supplier uuidGenerator, final JsonSecretsProcessor secretsProcessor, final ConfigurationUpdate configurationUpdate) { this.configRepository = configRepository; this.validator = integrationSchemaValidation; - this.specFetcher = specFetcher; this.connectionsHandler = connectionsHandler; this.uuidGenerator = uuidGenerator; this.configurationUpdate = configurationUpdate; @@ -58,16 +54,14 @@ public class SourceHandler { public SourceHandler(final ConfigRepository configRepository, final JsonSchemaValidator integrationSchemaValidation, - final SpecFetcher specFetcher, final ConnectionsHandler connectionsHandler) { this( configRepository, integrationSchemaValidation, - specFetcher, connectionsHandler, UUID::randomUUID, new JsonSecretsProcessor(), - new ConfigurationUpdate(configRepository, specFetcher)); + new ConfigurationUpdate(configRepository)); } public SourceRead createSource(final SourceCreate sourceCreate) @@ -203,9 +197,8 @@ public void deleteSource(final SourceRead source) private SourceRead buildSourceRead(final UUID sourceId) throws ConfigNotFoundException, IOException, JsonValidationException { // read configuration from db - final StandardSourceDefinition sourceDef = configRepository - .getSourceDefinitionFromSource(sourceId); - final ConnectorSpecification spec = specFetcher.getSpec(sourceDef); + final StandardSourceDefinition sourceDef = configRepository.getSourceDefinitionFromSource(sourceId); + final ConnectorSpecification spec = sourceDef.getSpec(); return buildSourceRead(sourceId, spec); } @@ -235,12 +228,7 @@ private ConnectorSpecification getSpecFromSourceId(final UUID sourceId) private ConnectorSpecification getSpecFromSourceDefinitionId(final UUID sourceDefId) throws IOException, JsonValidationException, ConfigNotFoundException { final StandardSourceDefinition sourceDef = configRepository.getStandardSourceDefinition(sourceDefId); - return getSpecFromSourceDefinitionId(specFetcher, sourceDef); - } - - public static ConnectorSpecification getSpecFromSourceDefinitionId(final SpecFetcher specFetcher, final StandardSourceDefinition sourceDefinition) - throws IOException, ConfigNotFoundException { - return specFetcher.getSpec(sourceDefinition); + return sourceDef.getSpec(); } private void persistSourceConnection(final String name, diff --git a/airbyte-server/src/test/java/io/airbyte/server/ConfigDumpImporterTest.java b/airbyte-server/src/test/java/io/airbyte/server/ConfigDumpImporterTest.java index 28700a826041..22c997a7a986 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/ConfigDumpImporterTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/ConfigDumpImporterTest.java @@ -29,7 +29,6 @@ import io.airbyte.scheduler.persistence.DefaultJobPersistence; import io.airbyte.scheduler.persistence.JobPersistence; import io.airbyte.scheduler.persistence.WorkspaceHelper; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.validation.json.JsonSchemaValidator; import io.airbyte.validation.json.JsonValidationException; import java.io.File; @@ -63,14 +62,10 @@ public void setup() throws IOException, JsonValidationException, ConfigNotFoundE final JobPersistence jobPersistence = mock(JobPersistence.class); final WorkspaceHelper workspaceHelper = mock(WorkspaceHelper.class); - final SpecFetcher specFetcher = mock(SpecFetcher.class); - emptyConnectorSpec = mock(ConnectorSpecification.class); - when(emptyConnectorSpec.getConnectionSpecification()).thenReturn(Jsons.emptyObject()); - when(specFetcher.getSpec(any(StandardSourceDefinition.class))).thenReturn(emptyConnectorSpec); - when(specFetcher.getSpec(any(StandardDestinationDefinition.class))).thenReturn(emptyConnectorSpec); + emptyConnectorSpec = new ConnectorSpecification().withConnectionSpecification(Jsons.emptyObject()); configDumpImporter = - new ConfigDumpImporter(configRepository, jobPersistence, workspaceHelper, mock(JsonSchemaValidator.class), specFetcher, true); + new ConfigDumpImporter(configRepository, jobPersistence, workspaceHelper, mock(JsonSchemaValidator.class), true); configDumpExporter = new ConfigDumpExporter(configRepository, jobPersistence, workspaceHelper); workspaceId = UUID.randomUUID(); @@ -83,7 +78,7 @@ public void setup() throws IOException, JsonValidationException, ConfigNotFoundE .withDocumentationUrl("http://doc") .withIcon("hello") .withDockerImageTag("dev") - .withSpec(new ConnectorSpecification()); + .withSpec(emptyConnectorSpec); sourceConnection = new SourceConnection() .withSourceId(UUID.randomUUID()) .withSourceDefinitionId(standardSourceDefinition.getSourceDefinitionId()) @@ -105,7 +100,7 @@ public void setup() throws IOException, JsonValidationException, ConfigNotFoundE .withDocumentationUrl("http://doc") .withIcon("hello") .withDockerImageTag("dev") - .withSpec(new ConnectorSpecification()); + .withSpec(emptyConnectorSpec); destinationConnection = new DestinationConnection() .withDestinationId(UUID.randomUUID()) .withDestinationDefinitionId(standardDestinationDefinition.getDestinationDefinitionId()) diff --git a/airbyte-server/src/test/java/io/airbyte/server/ConnectorDefinitionSpecBackfillerTest.java b/airbyte-server/src/test/java/io/airbyte/server/ConnectorDefinitionSpecBackfillerTest.java deleted file mode 100644 index 23e6045943f2..000000000000 --- a/airbyte-server/src/test/java/io/airbyte/server/ConnectorDefinitionSpecBackfillerTest.java +++ /dev/null @@ -1,296 +0,0 @@ -/* - * Copyright (c) 2021 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.server; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyMap; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import io.airbyte.analytics.TrackingClient; -import io.airbyte.commons.json.Jsons; -import io.airbyte.config.ConfigSchema; -import io.airbyte.config.Configs; -import io.airbyte.config.Configs.WorkerEnvironment; -import io.airbyte.config.DestinationConnection; -import io.airbyte.config.JobConfig.ConfigType; -import io.airbyte.config.SourceConnection; -import io.airbyte.config.StandardDestinationDefinition; -import io.airbyte.config.StandardSourceDefinition; -import io.airbyte.config.StandardWorkspace; -import io.airbyte.config.helpers.LogConfigs; -import io.airbyte.config.init.YamlSeedConfigPersistence; -import io.airbyte.config.persistence.ConfigNotFoundException; -import io.airbyte.config.persistence.ConfigRepository; -import io.airbyte.config.persistence.DatabaseConfigPersistence; -import io.airbyte.protocol.models.ConnectorSpecification; -import io.airbyte.scheduler.client.SynchronousJobMetadata; -import io.airbyte.scheduler.client.SynchronousResponse; -import io.airbyte.scheduler.client.SynchronousSchedulerClient; -import io.airbyte.validation.json.JsonValidationException; -import java.io.IOException; -import java.net.URI; -import java.nio.file.Path; -import java.time.Instant; -import java.util.List; -import java.util.Set; -import java.util.UUID; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -class ConnectorDefinitionSpecBackfillerTest { - - private static final String SOURCE_DEF_NAME = "source-definition"; - private static final String DEST_DEF_NAME = "destination-definition"; - private static final String SOURCE_DOCKER_REPO = "docker-repo/source"; - private static final String DEST_DOCKER_REPO = "docker-repo/destination"; - private static final String DOCKER_IMAGE_TAG = "tag"; - private static final String FAILED_SPEC_BACKFILL_ACTION = "failed_spec_backfill_major_bump"; - private static final StandardWorkspace WORKSPACE = new StandardWorkspace().withWorkspaceId(UUID.randomUUID()); - - private ConfigRepository configRepository; - private DatabaseConfigPersistence database; - private YamlSeedConfigPersistence seed; - private TrackingClient trackingClient; - private SynchronousSchedulerClient schedulerClient; - private Configs configs; - - @BeforeEach - void setup() throws IOException, JsonValidationException { - configRepository = mock(ConfigRepository.class); - when(configRepository.listStandardWorkspaces(true)).thenReturn(List.of(WORKSPACE)); - - database = mock(DatabaseConfigPersistence.class); - seed = mock(YamlSeedConfigPersistence.class); - trackingClient = mock(TrackingClient.class); - schedulerClient = mock(SynchronousSchedulerClient.class); - configs = mock(Configs.class); - when(configs.getWorkerEnvironment()).thenReturn(WorkerEnvironment.DOCKER); - when(configs.getLogConfigs()).thenReturn(mock(LogConfigs.class)); - } - - @Test - public void testBackfillSpecSuccessful() throws JsonValidationException, IOException, ConfigNotFoundException { - final StandardSourceDefinition sourceDef = new StandardSourceDefinition().withSourceDefinitionId(UUID.randomUUID()) - .withDockerRepository(SOURCE_DOCKER_REPO) - .withDockerImageTag(DOCKER_IMAGE_TAG) - .withName(SOURCE_DEF_NAME); - final StandardDestinationDefinition destDef = new StandardDestinationDefinition().withDestinationDefinitionId(UUID.randomUUID()) - .withDockerRepository(DEST_DOCKER_REPO) - .withDockerImageTag(DOCKER_IMAGE_TAG) - .withName(DEST_DEF_NAME); - - when(configRepository.listStandardSourceDefinitions()).thenReturn(List.of(sourceDef)); - when(configRepository.listStandardDestinationDefinitions()).thenReturn(List.of(destDef)); - // source def is in use but not in seed, should be backfilled - when(database.getInUseConnectorDockerImageNames()).thenReturn(Set.of(SOURCE_DOCKER_REPO)); - when(seed.listConfigs(ConfigSchema.STANDARD_SOURCE_DEFINITION, StandardSourceDefinition.class)).thenReturn(List.of()); - // dest def is not in use but is in seed, should be backfilled - when(seed.listConfigs(ConfigSchema.STANDARD_DESTINATION_DEFINITION, StandardDestinationDefinition.class)).thenReturn(List.of(destDef)); - - final ConnectorSpecification sourceSpec = new ConnectorSpecification().withDocumentationUrl(URI.create("http://source.org")); - final ConnectorSpecification destSpec = new ConnectorSpecification().withDocumentationUrl(URI.create("http://dest.org")); - - final SynchronousResponse successfulSourceResponse = new SynchronousResponse<>( - sourceSpec, - mockJobMetadata(true)); - final SynchronousResponse successfulDestResponse = new SynchronousResponse<>( - destSpec, - mockJobMetadata(true)); - - final SynchronousSchedulerClient schedulerClient = mock(SynchronousSchedulerClient.class); - when(schedulerClient.createGetSpecJob(SOURCE_DOCKER_REPO + ":" + DOCKER_IMAGE_TAG)).thenReturn(successfulSourceResponse); - when(schedulerClient.createGetSpecJob(DEST_DOCKER_REPO + ":" + DOCKER_IMAGE_TAG)).thenReturn(successfulDestResponse); - - ConnectorDefinitionSpecBackfiller.migrateAllDefinitionsToContainSpec( - configRepository, - database, - seed, - schedulerClient, - trackingClient, - configs); - - final StandardSourceDefinition expectedSourceDef = Jsons.clone(sourceDef).withSpec(sourceSpec); - final StandardDestinationDefinition expectedDestDef = Jsons.clone(destDef).withSpec(destSpec); - verify(configRepository, times(1)).writeStandardSourceDefinition(expectedSourceDef); - verify(configRepository, times(1)).writeStandardDestinationDefinition(expectedDestDef); - verify(configRepository, never()).deleteSourceDefinitionAndAssociations(any()); - verify(configRepository, never()).deleteDestinationDefinitionAndAssociations(any()); - } - - @Test - public void testDeleteUnusedDefinitionsNotInSeed() throws JsonValidationException, IOException, ConfigNotFoundException { - final StandardSourceDefinition sourceDef = new StandardSourceDefinition().withSourceDefinitionId(UUID.randomUUID()) - .withDockerRepository(SOURCE_DOCKER_REPO) - .withDockerImageTag(DOCKER_IMAGE_TAG) - .withName(SOURCE_DEF_NAME); - final StandardDestinationDefinition destDef = new StandardDestinationDefinition().withDestinationDefinitionId(UUID.randomUUID()) - .withDockerRepository(DEST_DOCKER_REPO) - .withDockerImageTag(DOCKER_IMAGE_TAG) - .withName(DEST_DEF_NAME); - - when(configRepository.listStandardSourceDefinitions()).thenReturn(List.of(sourceDef)); - when(configRepository.listStandardDestinationDefinitions()).thenReturn(List.of(destDef)); - // both source and destination definitions are not in use and are not in the seed, should be deleted - when(database.getInUseConnectorDockerImageNames()).thenReturn(Set.of()); - when(seed.listConfigs(ConfigSchema.STANDARD_SOURCE_DEFINITION, StandardSourceDefinition.class)).thenReturn(List.of()); - when(seed.listConfigs(ConfigSchema.STANDARD_DESTINATION_DEFINITION, StandardDestinationDefinition.class)).thenReturn(List.of()); - - when(database.listConfigs(ConfigSchema.SOURCE_CONNECTION, SourceConnection.class)).thenReturn(List.of()); - when(database.listConfigs(ConfigSchema.DESTINATION_CONNECTION, DestinationConnection.class)).thenReturn(List.of()); - - ConnectorDefinitionSpecBackfiller.migrateAllDefinitionsToContainSpec( - configRepository, - database, - seed, - schedulerClient, - trackingClient, - configs); - - verify(configRepository, never()).writeStandardSourceDefinition(any()); - verify(configRepository, never()).writeStandardDestinationDefinition(any()); - verify(configRepository, times(1)).deleteSourceDefinitionAndAssociations(sourceDef.getSourceDefinitionId()); - verify(configRepository, times(1)).deleteDestinationDefinitionAndAssociations(destDef.getDestinationDefinitionId()); - } - - @Test - public void testBackfillSpecFailureNonForceUpgrade() throws JsonValidationException, IOException, ConfigNotFoundException { - final StandardSourceDefinition sourceDef = new StandardSourceDefinition().withSourceDefinitionId(UUID.randomUUID()) - .withDockerRepository(SOURCE_DOCKER_REPO) - .withDockerImageTag(DOCKER_IMAGE_TAG) - .withName(SOURCE_DEF_NAME); - final StandardDestinationDefinition destDef = new StandardDestinationDefinition().withDestinationDefinitionId(UUID.randomUUID()) - .withDockerRepository(DEST_DOCKER_REPO) - .withDockerImageTag(DOCKER_IMAGE_TAG) - .withName(DEST_DEF_NAME); - - when(configRepository.listStandardSourceDefinitions()).thenReturn(List.of(sourceDef)); - when(configRepository.listStandardDestinationDefinitions()).thenReturn(List.of(destDef)); - // both source and destination definitions are in use, so should be backfilled - when(database.getInUseConnectorDockerImageNames()).thenReturn(Set.of(SOURCE_DOCKER_REPO, DEST_DOCKER_REPO)); - - final SynchronousResponse failureSourceResponse = new SynchronousResponse<>( - null, - mockJobMetadata(false)); - final SynchronousResponse failureDestResponse = new SynchronousResponse<>( - null, - mockJobMetadata(false)); - - when(schedulerClient.createGetSpecJob(SOURCE_DOCKER_REPO + ":" + DOCKER_IMAGE_TAG)).thenReturn(failureSourceResponse); - when(schedulerClient.createGetSpecJob(DEST_DOCKER_REPO + ":" + DOCKER_IMAGE_TAG)).thenReturn(failureDestResponse); - - // do not force upgrade, should fail with an error instead - when(configs.getVersion32ForceUpgrade()).thenReturn(false); - - assertThatThrownBy(() -> ConnectorDefinitionSpecBackfiller.migrateAllDefinitionsToContainSpec( - configRepository, - database, - seed, - schedulerClient, - trackingClient, - configs)) - .isInstanceOf(RuntimeException.class) - .hasMessageContaining( - "Specs could not be retrieved for the following connector images: [" - + SOURCE_DOCKER_REPO + ":" + DOCKER_IMAGE_TAG + ", " + DEST_DOCKER_REPO + ":" + DOCKER_IMAGE_TAG + "]"); - - verify(configRepository, never()).writeStandardSourceDefinition(any()); - verify(configRepository, never()).writeStandardDestinationDefinition(any()); - verify(configRepository, never()).deleteSourceDefinitionAndAssociations(any()); - verify(configRepository, never()).deleteDestinationDefinitionAndAssociations(any()); - - verify(trackingClient, times(2)).track(eq(WORKSPACE.getWorkspaceId()), eq(FAILED_SPEC_BACKFILL_ACTION), anyMap()); - } - - @Test - public void testBackfillSpecFailureForceUpgrade() throws JsonValidationException, IOException, ConfigNotFoundException { - final StandardSourceDefinition sourceDef = new StandardSourceDefinition().withSourceDefinitionId(UUID.randomUUID()) - .withDockerRepository(SOURCE_DOCKER_REPO) - .withDockerImageTag(DOCKER_IMAGE_TAG) - .withName(SOURCE_DEF_NAME); - final StandardDestinationDefinition destDef = new StandardDestinationDefinition().withDestinationDefinitionId(UUID.randomUUID()) - .withDockerRepository(DEST_DOCKER_REPO) - .withDockerImageTag(DOCKER_IMAGE_TAG) - .withName(DEST_DEF_NAME); - - when(configRepository.listStandardSourceDefinitions()).thenReturn(List.of(sourceDef)); - when(configRepository.listStandardDestinationDefinitions()).thenReturn(List.of(destDef)); - // both source and destination definitions are in use, so should be backfilled - when(database.getInUseConnectorDockerImageNames()).thenReturn(Set.of(SOURCE_DOCKER_REPO, DEST_DOCKER_REPO)); - - final SynchronousResponse failureSourceResponse = new SynchronousResponse<>( - null, - mockJobMetadata(false)); - final SynchronousResponse failureDestResponse = new SynchronousResponse<>( - null, - mockJobMetadata(false)); - - when(schedulerClient.createGetSpecJob(SOURCE_DOCKER_REPO + ":" + DOCKER_IMAGE_TAG)).thenReturn(failureSourceResponse); - when(schedulerClient.createGetSpecJob(DEST_DOCKER_REPO + ":" + DOCKER_IMAGE_TAG)).thenReturn(failureDestResponse); - - // force upgrade, specs should be deleted - when(configs.getVersion32ForceUpgrade()).thenReturn(true); - - ConnectorDefinitionSpecBackfiller.migrateAllDefinitionsToContainSpec( - configRepository, - database, - seed, - schedulerClient, - trackingClient, - configs); - - verify(configRepository, never()).writeStandardSourceDefinition(any()); - verify(configRepository, never()).writeStandardDestinationDefinition(any()); - verify(configRepository, times(1)).deleteSourceDefinitionAndAssociations(sourceDef.getSourceDefinitionId()); - verify(configRepository, times(1)).deleteDestinationDefinitionAndAssociations(destDef.getDestinationDefinitionId()); - - verify(trackingClient, times(2)).track(eq(WORKSPACE.getWorkspaceId()), eq(FAILED_SPEC_BACKFILL_ACTION), anyMap()); - } - - @Test - public void testSpecAlreadyExists() throws JsonValidationException, IOException, ConfigNotFoundException { - final ConnectorSpecification sourceSpec = new ConnectorSpecification().withDocumentationUrl(URI.create("http://source.org")); - final ConnectorSpecification destSpec = new ConnectorSpecification().withDocumentationUrl(URI.create("http://dest.org")); - final StandardSourceDefinition sourceDef = new StandardSourceDefinition().withDockerRepository(SOURCE_DOCKER_REPO) - .withDockerImageTag(DOCKER_IMAGE_TAG).withSpec(sourceSpec); - final StandardDestinationDefinition destDef = new StandardDestinationDefinition().withDockerRepository(DEST_DOCKER_REPO) - .withDockerImageTag(DOCKER_IMAGE_TAG).withSpec(destSpec); - - when(configRepository.listStandardSourceDefinitions()).thenReturn(List.of(sourceDef)); - when(configRepository.listStandardDestinationDefinitions()).thenReturn(List.of(destDef)); - - ConnectorDefinitionSpecBackfiller.migrateAllDefinitionsToContainSpec( - configRepository, - database, - seed, - schedulerClient, - trackingClient, - configs); - - verify(schedulerClient, never()).createGetSpecJob(any()); - verify(configRepository, never()).writeStandardSourceDefinition(any()); - verify(configRepository, never()).writeStandardDestinationDefinition(any()); - verify(configRepository, never()).deleteSourceDefinitionAndAssociations(any()); - verify(configRepository, never()).deleteDestinationDefinitionAndAssociations(any()); - } - - private SynchronousJobMetadata mockJobMetadata(final boolean succeeded) { - final long now = Instant.now().toEpochMilli(); - return new SynchronousJobMetadata( - UUID.randomUUID(), - ConfigType.GET_SPEC, - UUID.randomUUID(), - now, - now, - succeeded, - Path.of("path", "to", "logs")); - } - -} diff --git a/airbyte-server/src/test/java/io/airbyte/server/apis/ConfigurationApiTest.java b/airbyte-server/src/test/java/io/airbyte/server/apis/ConfigurationApiTest.java index b71d05eda52d..e0e76160fd2c 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/apis/ConfigurationApiTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/apis/ConfigurationApiTest.java @@ -17,8 +17,8 @@ import io.airbyte.config.persistence.ConfigPersistence; import io.airbyte.config.persistence.ConfigRepository; import io.airbyte.db.Database; -import io.airbyte.scheduler.client.CachingSynchronousSchedulerClient; import io.airbyte.scheduler.client.SchedulerJobClient; +import io.airbyte.scheduler.client.SynchronousSchedulerClient; import io.airbyte.scheduler.persistence.JobPersistence; import io.temporal.serviceclient.WorkflowServiceStubs; import java.net.http.HttpClient; @@ -38,7 +38,7 @@ void testImportDefinitions() { mock(JobPersistence.class), mock(ConfigPersistence.class), mock(SchedulerJobClient.class), - mock(CachingSynchronousSchedulerClient.class), + mock(SynchronousSchedulerClient.class), mock(FileTtlManager.class), mock(WorkflowServiceStubs.class), mock(Database.class), diff --git a/airbyte-server/src/test/java/io/airbyte/server/converters/ConfigurationUpdateTest.java b/airbyte-server/src/test/java/io/airbyte/server/converters/ConfigurationUpdateTest.java index e5173a306368..0bba7563b889 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/converters/ConfigurationUpdateTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/converters/ConfigurationUpdateTest.java @@ -49,7 +49,8 @@ class ConfigurationUpdateTest { .build()); private static final StandardSourceDefinition SOURCE_DEFINITION = new StandardSourceDefinition() .withDockerRepository(IMAGE_REPOSITORY) - .withDockerImageTag(IMAGE_TAG); + .withDockerImageTag(IMAGE_TAG) + .withSpec(CONNECTOR_SPECIFICATION); private static final SourceConnection ORIGINAL_SOURCE_CONNECTION = new SourceConnection() .withSourceId(UUID1) .withSourceDefinitionId(UUID2) @@ -60,7 +61,8 @@ class ConfigurationUpdateTest { .withConfiguration(NEW_CONFIGURATION); private static final StandardDestinationDefinition DESTINATION_DEFINITION = new StandardDestinationDefinition() .withDockerRepository(IMAGE_REPOSITORY) - .withDockerImageTag(IMAGE_TAG); + .withDockerImageTag(IMAGE_TAG) + .withSpec(CONNECTOR_SPECIFICATION); private static final DestinationConnection ORIGINAL_DESTINATION_CONNECTION = new DestinationConnection() .withDestinationId(UUID1) .withDestinationDefinitionId(UUID2) @@ -71,24 +73,21 @@ class ConfigurationUpdateTest { .withConfiguration(NEW_CONFIGURATION); private ConfigRepository configRepository; - private SpecFetcher specFetcher; private JsonSecretsProcessor secretsProcessor; private ConfigurationUpdate configurationUpdate; @BeforeEach void setup() { configRepository = mock(ConfigRepository.class); - specFetcher = mock(SpecFetcher.class); secretsProcessor = mock(JsonSecretsProcessor.class); - configurationUpdate = new ConfigurationUpdate(configRepository, specFetcher, secretsProcessor); + configurationUpdate = new ConfigurationUpdate(configRepository, secretsProcessor); } @Test void testSourceUpdate() throws JsonValidationException, IOException, ConfigNotFoundException { when(configRepository.getSourceConnectionWithSecrets(UUID1)).thenReturn(ORIGINAL_SOURCE_CONNECTION); when(configRepository.getStandardSourceDefinition(UUID2)).thenReturn(SOURCE_DEFINITION); - when(specFetcher.getSpec(SOURCE_DEFINITION)).thenReturn(CONNECTOR_SPECIFICATION); when(secretsProcessor.copySecrets(ORIGINAL_CONFIGURATION, NEW_CONFIGURATION, SPEC)).thenReturn(NEW_CONFIGURATION); final SourceConnection actual = configurationUpdate.source(UUID1, ORIGINAL_SOURCE_CONNECTION.getName(), NEW_CONFIGURATION); @@ -100,7 +99,6 @@ void testSourceUpdate() throws JsonValidationException, IOException, ConfigNotFo void testDestinationUpdate() throws JsonValidationException, IOException, ConfigNotFoundException { when(configRepository.getDestinationConnectionWithSecrets(UUID1)).thenReturn(ORIGINAL_DESTINATION_CONNECTION); when(configRepository.getStandardDestinationDefinition(UUID2)).thenReturn(DESTINATION_DEFINITION); - when(specFetcher.getSpec(DESTINATION_DEFINITION)).thenReturn(CONNECTOR_SPECIFICATION); when(secretsProcessor.copySecrets(ORIGINAL_CONFIGURATION, NEW_CONFIGURATION, SPEC)).thenReturn(NEW_CONFIGURATION); final DestinationConnection actual = configurationUpdate.destination(UUID1, ORIGINAL_DESTINATION_CONNECTION.getName(), NEW_CONFIGURATION); diff --git a/airbyte-server/src/test/java/io/airbyte/server/converters/SpecFetcherTest.java b/airbyte-server/src/test/java/io/airbyte/server/converters/SpecFetcherTest.java deleted file mode 100644 index 21dc94fdafe3..000000000000 --- a/airbyte-server/src/test/java/io/airbyte/server/converters/SpecFetcherTest.java +++ /dev/null @@ -1,126 +0,0 @@ -/* - * Copyright (c) 2021 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.server.converters; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import com.google.common.collect.ImmutableMap; -import io.airbyte.commons.json.Jsons; -import io.airbyte.config.JobConfig.ConfigType; -import io.airbyte.config.StandardDestinationDefinition; -import io.airbyte.config.StandardSourceDefinition; -import io.airbyte.protocol.models.ConnectorSpecification; -import io.airbyte.scheduler.client.SynchronousResponse; -import io.airbyte.scheduler.client.SynchronousSchedulerClient; -import java.io.IOException; -import java.util.Optional; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -class SpecFetcherTest { - - private static final String DOCKER_REPOSITORY = "foo"; - private static final String DOCKER_IMAGE_TAG = "bar"; - private static final String IMAGE_NAME = DOCKER_REPOSITORY + ":" + DOCKER_IMAGE_TAG; - - private SynchronousSchedulerClient schedulerJobClient; - private SynchronousResponse response; - private ConnectorSpecification connectorSpecification; - private StandardSourceDefinition sourceDefinition; - private StandardDestinationDefinition destinationDefinition; - - @SuppressWarnings("unchecked") - @BeforeEach - void setup() { - schedulerJobClient = mock(SynchronousSchedulerClient.class); - response = mock(SynchronousResponse.class); - connectorSpecification = new ConnectorSpecification().withConnectionSpecification(Jsons.jsonNode(ImmutableMap.of("foo", "bar"))); - sourceDefinition = new StandardSourceDefinition().withDockerRepository(DOCKER_REPOSITORY).withDockerImageTag(DOCKER_IMAGE_TAG); - destinationDefinition = new StandardDestinationDefinition().withDockerRepository(DOCKER_REPOSITORY).withDockerImageTag(DOCKER_IMAGE_TAG); - } - - @Test - void testGetSpecFromDockerImageSuccess() throws IOException { - when(schedulerJobClient.createGetSpecJob(IMAGE_NAME)).thenReturn(response); - when(response.isSuccess()).thenReturn(true); - when(response.getOutput()).thenReturn(connectorSpecification); - - final SpecFetcher specFetcher = new SpecFetcher(schedulerJobClient); - assertEquals(connectorSpecification, specFetcher.getSpec(IMAGE_NAME)); - } - - @Test - void testGetSpecFromDockerImageFail() throws IOException { - when(schedulerJobClient.createGetSpecJob(IMAGE_NAME)).thenReturn(response); - when(response.isSuccess()).thenReturn(false); - - final SpecFetcher specFetcher = new SpecFetcher(schedulerJobClient); - assertThrows(IllegalStateException.class, () -> specFetcher.getSpec(IMAGE_NAME)); - } - - @Test - void testGetSpecFromSourceDefinitionNotNull() throws IOException { - final StandardSourceDefinition sourceDefinitionWithSpec = Jsons.clone(sourceDefinition).withSpec(connectorSpecification); - - final SpecFetcher specFetcher = new SpecFetcher(schedulerJobClient); - assertEquals(connectorSpecification, specFetcher.getSpec(sourceDefinitionWithSpec)); - } - - @Test - void testGetSpecFromSourceDefinitionNull() throws IOException { - when(schedulerJobClient.createGetSpecJob(IMAGE_NAME)).thenReturn(response); - when(response.isSuccess()).thenReturn(true); - when(response.getOutput()).thenReturn(connectorSpecification); - - final SpecFetcher specFetcher = new SpecFetcher(schedulerJobClient); - assertEquals(connectorSpecification, specFetcher.getSpec(sourceDefinition)); - } - - @Test - void testGetSpecFromDestinationDefinitionNotNull() throws IOException { - final StandardDestinationDefinition destinationDefinitionWithSpec = Jsons.clone(destinationDefinition).withSpec(connectorSpecification); - - final SpecFetcher specFetcher = new SpecFetcher(schedulerJobClient); - assertEquals(connectorSpecification, specFetcher.getSpec(destinationDefinitionWithSpec)); - } - - @Test - void testGetSpecFromDestinationDefinitionNull() throws IOException { - when(schedulerJobClient.createGetSpecJob(IMAGE_NAME)).thenReturn(response); - when(response.isSuccess()).thenReturn(true); - when(response.getOutput()).thenReturn(connectorSpecification); - - final SpecFetcher specFetcher = new SpecFetcher(schedulerJobClient); - assertEquals(connectorSpecification, specFetcher.getSpec(destinationDefinition)); - } - - @Test - void testGetSpecJobResponseFromSourceReturnsMockedJobMetadata() throws IOException { - final StandardSourceDefinition sourceDefinitionWithSpec = Jsons.clone(sourceDefinition).withSpec(connectorSpecification); - - final SpecFetcher specFetcher = new SpecFetcher(schedulerJobClient); - final SynchronousResponse response = specFetcher.getSpecJobResponse(sourceDefinitionWithSpec); - - assertEquals(ConfigType.GET_SPEC, response.getMetadata().getConfigType()); - assertEquals(Optional.empty(), response.getMetadata().getConfigId()); - assertEquals(connectorSpecification, response.getOutput()); - } - - @Test - void testGetSpecJobResponseFromDestinationReturnsMockedJobMetadata() throws IOException { - final StandardDestinationDefinition destinationDefinitionWithSpec = Jsons.clone(destinationDefinition).withSpec(connectorSpecification); - - final SpecFetcher specFetcher = new SpecFetcher(schedulerJobClient); - final SynchronousResponse response = specFetcher.getSpecJobResponse(destinationDefinitionWithSpec); - - assertEquals(ConfigType.GET_SPEC, response.getMetadata().getConfigType()); - assertEquals(Optional.empty(), response.getMetadata().getConfigId()); - assertEquals(connectorSpecification, response.getOutput()); - } - -} diff --git a/airbyte-server/src/test/java/io/airbyte/server/handlers/ArchiveHandlerTest.java b/airbyte-server/src/test/java/io/airbyte/server/handlers/ArchiveHandlerTest.java index 396820fed21d..5e3b4ca0d9c0 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/handlers/ArchiveHandlerTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/handlers/ArchiveHandlerTest.java @@ -7,7 +7,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -24,7 +23,6 @@ import io.airbyte.config.ConfigSchema; import io.airbyte.config.DestinationConnection; import io.airbyte.config.SourceConnection; -import io.airbyte.config.StandardDestinationDefinition; import io.airbyte.config.StandardSourceDefinition; import io.airbyte.config.StandardWorkspace; import io.airbyte.config.init.YamlSeedConfigPersistence; @@ -38,7 +36,6 @@ import io.airbyte.scheduler.persistence.DefaultJobPersistence; import io.airbyte.scheduler.persistence.JobPersistence; import io.airbyte.scheduler.persistence.WorkspaceHelper; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.validation.json.JsonValidationException; import java.io.File; import java.io.IOException; @@ -60,6 +57,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.PostgreSQLContainer; +import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; import org.testcontainers.shaded.org.apache.commons.io.FileUtils; public class ArchiveHandlerTest { @@ -116,12 +114,6 @@ public void setup() throws Exception { jobPersistence.setVersion(VERSION.serialize()); - final SpecFetcher specFetcher = mock(SpecFetcher.class); - final ConnectorSpecification emptyConnectorSpec = mock(ConnectorSpecification.class); - when(emptyConnectorSpec.getConnectionSpecification()).thenReturn(Jsons.emptyObject()); - when(specFetcher.getSpec(any(StandardSourceDefinition.class))).thenReturn(emptyConnectorSpec); - when(specFetcher.getSpec(any(StandardDestinationDefinition.class))).thenReturn(emptyConnectorSpec); - archiveHandler = new ArchiveHandler( VERSION, configRepository, @@ -129,7 +121,6 @@ public void setup() throws Exception { YamlSeedConfigPersistence.getDefault(), new WorkspaceHelper(configRepository, jobPersistence), new NoOpFileTtlManager(), - specFetcher, true); } @@ -295,17 +286,17 @@ private void setupTestData(final UUID workspaceId) throws JsonValidationExceptio configPersistence.writeConfig(ConfigSchema.SOURCE_CONNECTION, sourceid.toString(), new SourceConnection() .withSourceId(sourceid) .withWorkspaceId(workspaceId) - .withSourceDefinitionId(configRepository.listStandardSourceDefinitions().get(0).getSourceDefinitionId()) + .withSourceDefinitionId(UUID.fromString("ef69ef6e-aa7f-4af1-a01d-ef775033524e")) // GitHub source definition .withName("test-source") - .withConfiguration(Jsons.emptyObject()) + .withConfiguration(Jsons.jsonNode(ImmutableMap.of("start_date", "2021-03-01T00:00:00Z", "repository", "airbytehq/airbyte"))) .withTombstone(false)); final UUID destinationId = UUID.randomUUID(); configPersistence.writeConfig(ConfigSchema.DESTINATION_CONNECTION, destinationId.toString(), new DestinationConnection() .withDestinationId(destinationId) .withWorkspaceId(workspaceId) - .withDestinationDefinitionId(configRepository.listStandardDestinationDefinitions().get(0).getDestinationDefinitionId()) + .withDestinationDefinitionId(UUID.fromString("079d5540-f236-4294-ba7c-ade8fd918496")) // BigQuery destination definition .withName("test-destination") - .withConfiguration(Jsons.emptyObject()) + .withConfiguration(Jsons.jsonNode(ImmutableMap.of("project_id", "project", "dataset_id", "dataset"))) .withTombstone(false)); } diff --git a/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationDefinitionsHandlerTest.java b/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationDefinitionsHandlerTest.java index 519cfb69188e..84166be09f19 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationDefinitionsHandlerTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationDefinitionsHandlerTest.java @@ -25,9 +25,9 @@ import io.airbyte.config.persistence.ConfigNotFoundException; import io.airbyte.config.persistence.ConfigRepository; import io.airbyte.protocol.models.ConnectorSpecification; -import io.airbyte.scheduler.client.CachingSynchronousSchedulerClient; import io.airbyte.scheduler.client.SynchronousJobMetadata; import io.airbyte.scheduler.client.SynchronousResponse; +import io.airbyte.scheduler.client.SynchronousSchedulerClient; import io.airbyte.server.services.AirbyteGithubStore; import io.airbyte.validation.json.JsonValidationException; import java.io.IOException; @@ -47,7 +47,7 @@ class DestinationDefinitionsHandlerTest { private StandardDestinationDefinition destination; private DestinationDefinitionsHandler destinationHandler; private Supplier uuidSupplier; - private CachingSynchronousSchedulerClient schedulerSynchronousClient; + private SynchronousSchedulerClient schedulerSynchronousClient; private AirbyteGithubStore githubStore; @SuppressWarnings("unchecked") @@ -56,7 +56,7 @@ void setUp() { configRepository = mock(ConfigRepository.class); uuidSupplier = mock(Supplier.class); destination = generateDestination(); - schedulerSynchronousClient = spy(CachingSynchronousSchedulerClient.class); + schedulerSynchronousClient = spy(SynchronousSchedulerClient.class); githubStore = mock(AirbyteGithubStore.class); destinationHandler = @@ -188,7 +188,6 @@ void testUpdateDestination() throws ConfigNotFoundException, IOException, JsonVa assertEquals(newDockerImageTag, destinationRead.getDockerImageTag()); verify(schedulerSynchronousClient).createGetSpecJob(newImageName); verify(configRepository).writeStandardDestinationDefinition(updatedDestination); - verify(schedulerSynchronousClient).resetCache(); } @Nested diff --git a/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationHandlerTest.java b/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationHandlerTest.java index fee914ef03b4..d6c556d7cc6f 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationHandlerTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationHandlerTest.java @@ -34,7 +34,6 @@ import io.airbyte.config.persistence.split_secrets.JsonSecretsProcessor; import io.airbyte.protocol.models.ConnectorSpecification; import io.airbyte.server.converters.ConfigurationUpdate; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.server.helpers.ConnectionHelpers; import io.airbyte.server.helpers.ConnectorSpecificationHelpers; import io.airbyte.server.helpers.DestinationHelpers; @@ -55,7 +54,6 @@ class DestinationHandlerTest { private DestinationConnection destinationConnection; private DestinationHandler destinationHandler; private ConnectionsHandler connectionsHandler; - private SpecFetcher specFetcher; private ConfigurationUpdate configurationUpdate; private JsonSchemaValidator validator; private Supplier uuidGenerator; @@ -70,16 +68,18 @@ void setUp() throws IOException { validator = mock(JsonSchemaValidator.class); uuidGenerator = mock(Supplier.class); connectionsHandler = mock(ConnectionsHandler.class); - specFetcher = mock(SpecFetcher.class); configurationUpdate = mock(ConfigurationUpdate.class); secretsProcessor = mock(JsonSecretsProcessor.class); + connectorSpecification = ConnectorSpecificationHelpers.generateConnectorSpecification(); + standardDestinationDefinition = new StandardDestinationDefinition() .withDestinationDefinitionId(UUID.randomUUID()) .withName("db2") .withDockerRepository("thebestrepo") .withDockerImageTag("thelatesttag") - .withDocumentationUrl("https://wikipedia.org"); + .withDocumentationUrl("https://wikipedia.org") + .withSpec(connectorSpecification); imageName = DockerUtils.getTaggedImageName(standardDestinationDefinition.getDockerRepository(), standardDestinationDefinition.getDockerImageTag()); @@ -87,8 +87,6 @@ void setUp() throws IOException { final DestinationDefinitionIdRequestBody destinationDefinitionIdRequestBody = new DestinationDefinitionIdRequestBody().destinationDefinitionId( standardDestinationDefinition.getDestinationDefinitionId()); - connectorSpecification = ConnectorSpecificationHelpers.generateConnectorSpecification(); - destinationDefinitionSpecificationRead = new DestinationDefinitionSpecificationRead() .connectionSpecification(connectorSpecification.getConnectionSpecification()) .destinationDefinitionId(standardDestinationDefinition.getDestinationDefinitionId()) @@ -99,7 +97,7 @@ void setUp() throws IOException { destinationConnection = DestinationHelpers.generateDestination(standardDestinationDefinition.getDestinationDefinitionId()); destinationHandler = - new DestinationHandler(configRepository, validator, specFetcher, connectionsHandler, uuidGenerator, secretsProcessor, configurationUpdate); + new DestinationHandler(configRepository, validator, connectionsHandler, uuidGenerator, secretsProcessor, configurationUpdate); } @Test @@ -110,7 +108,6 @@ void testCreateDestination() throws JsonValidationException, ConfigNotFoundExcep .thenReturn(destinationConnection); when(configRepository.getStandardDestinationDefinition(standardDestinationDefinition.getDestinationDefinitionId())) .thenReturn(standardDestinationDefinition); - when(specFetcher.getSpec(standardDestinationDefinition)).thenReturn(connectorSpecification); when(secretsProcessor.maskSecrets(destinationConnection.getConfiguration(), destinationDefinitionSpecificationRead.getConnectionSpecification())) .thenReturn(destinationConnection.getConfiguration()); @@ -159,7 +156,6 @@ void testDeleteDestination() throws JsonValidationException, ConfigNotFoundExcep .thenReturn(expectedDestinationConnection); when(configRepository.getStandardDestinationDefinition(standardDestinationDefinition.getDestinationDefinitionId())) .thenReturn(standardDestinationDefinition); - when(specFetcher.getSpec(standardDestinationDefinition)).thenReturn(connectorSpecification); when(connectionsHandler.listConnectionsForWorkspace(workspaceIdRequestBody)).thenReturn(connectionReadList); destinationHandler.deleteDestination(destinationId); @@ -196,7 +192,6 @@ void testUpdateDestination() throws JsonValidationException, ConfigNotFoundExcep .thenReturn(standardDestinationDefinition); when(configRepository.getDestinationConnection(destinationConnection.getDestinationId())) .thenReturn(expectedDestinationConnection); - when(specFetcher.getSpec(standardDestinationDefinition)).thenReturn(connectorSpecification); when(configurationUpdate.destination(destinationConnection.getDestinationId(), updatedDestName, newConfiguration)) .thenReturn(expectedDestinationConnection); @@ -229,7 +224,6 @@ void testGetDestination() throws JsonValidationException, ConfigNotFoundExceptio when(configRepository.getDestinationConnection(destinationConnection.getDestinationId())).thenReturn(destinationConnection); when(configRepository.getStandardDestinationDefinition(standardDestinationDefinition.getDestinationDefinitionId())) .thenReturn(standardDestinationDefinition); - when(specFetcher.getSpec(standardDestinationDefinition)).thenReturn(connectorSpecification); final DestinationRead actualDestinationRead = destinationHandler.getDestination(destinationIdRequestBody); @@ -263,7 +257,6 @@ void testListDestinationForWorkspace() throws JsonValidationException, ConfigNot when(configRepository.getDestinationConnection(destinationConnection.getDestinationId())).thenReturn(destinationConnection); when(configRepository.listDestinationConnection()).thenReturn(Lists.newArrayList(destinationConnection)); - when(specFetcher.getSpec(standardDestinationDefinition)).thenReturn(connectorSpecification); when(configRepository.getStandardDestinationDefinition(standardDestinationDefinition.getDestinationDefinitionId())) .thenReturn(standardDestinationDefinition); when(secretsProcessor.maskSecrets(destinationConnection.getConfiguration(), destinationDefinitionSpecificationRead.getConnectionSpecification())) @@ -288,7 +281,6 @@ void testSearchDestinations() throws JsonValidationException, ConfigNotFoundExce when(configRepository.getDestinationConnection(destinationConnection.getDestinationId())).thenReturn(destinationConnection); when(configRepository.listDestinationConnection()).thenReturn(Lists.newArrayList(destinationConnection)); - when(specFetcher.getSpec(standardDestinationDefinition)).thenReturn(connectorSpecification); when(configRepository.getStandardDestinationDefinition(standardDestinationDefinition.getDestinationDefinitionId())) .thenReturn(standardDestinationDefinition); when(secretsProcessor.maskSecrets(destinationConnection.getConfiguration(), destinationDefinitionSpecificationRead.getConnectionSpecification())) diff --git a/airbyte-server/src/test/java/io/airbyte/server/handlers/OAuthHandlerTest.java b/airbyte-server/src/test/java/io/airbyte/server/handlers/OAuthHandlerTest.java index df0aa8297590..f2daaad4b335 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/handlers/OAuthHandlerTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/handlers/OAuthHandlerTest.java @@ -15,7 +15,6 @@ import io.airbyte.config.DestinationOAuthParameter; import io.airbyte.config.SourceOAuthParameter; import io.airbyte.config.persistence.ConfigRepository; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.validation.json.JsonValidationException; import java.io.IOException; import java.net.http.HttpClient; @@ -35,15 +34,13 @@ class OAuthHandlerTest { private OAuthHandler handler; private TrackingClient trackingClient; private HttpClient httpClient; - private SpecFetcher specFetcher; @BeforeEach public void init() { configRepository = Mockito.mock(ConfigRepository.class); trackingClient = mock(TrackingClient.class); httpClient = Mockito.mock(HttpClient.class); - specFetcher = mock(SpecFetcher.class); - handler = new OAuthHandler(configRepository, httpClient, trackingClient, specFetcher); + handler = new OAuthHandler(configRepository, httpClient, trackingClient); } @Test diff --git a/airbyte-server/src/test/java/io/airbyte/server/handlers/SchedulerHandlerTest.java b/airbyte-server/src/test/java/io/airbyte/server/handlers/SchedulerHandlerTest.java index 0eabedfff9b3..5caf2ba2e908 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/handlers/SchedulerHandlerTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/handlers/SchedulerHandlerTest.java @@ -23,11 +23,13 @@ import io.airbyte.api.model.ConnectionState; import io.airbyte.api.model.DestinationCoreConfig; import io.airbyte.api.model.DestinationDefinitionIdRequestBody; +import io.airbyte.api.model.DestinationDefinitionSpecificationRead; import io.airbyte.api.model.DestinationIdRequestBody; import io.airbyte.api.model.DestinationUpdate; import io.airbyte.api.model.JobInfoRead; import io.airbyte.api.model.SourceCoreConfig; import io.airbyte.api.model.SourceDefinitionIdRequestBody; +import io.airbyte.api.model.SourceDefinitionSpecificationRead; import io.airbyte.api.model.SourceDiscoverSchemaRead; import io.airbyte.api.model.SourceIdRequestBody; import io.airbyte.api.model.SourceUpdate; @@ -67,7 +69,6 @@ import io.airbyte.scheduler.persistence.JobPersistence; import io.airbyte.scheduler.persistence.job_factory.OAuthConfigSupplier; import io.airbyte.server.converters.ConfigurationUpdate; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.server.helpers.ConnectionHelpers; import io.airbyte.server.helpers.DestinationHelpers; import io.airbyte.server.helpers.SourceHelpers; @@ -112,7 +113,7 @@ class SchedulerHandlerTest { .withConfiguration(Jsons.emptyObject()) .withTombstone(false); - private static final ConnectorSpecification CONNECTION_SPECIFICATION = new ConnectorSpecification() + private static final ConnectorSpecification CONNECTOR_SPECIFICATION = new ConnectorSpecification() .withDocumentationUrl(Exceptions.toRuntime(() -> new URI("https://google.com"))) .withChangelogUrl(Exceptions.toRuntime(() -> new URI("https://google.com"))) .withConnectionSpecification(Jsons.jsonNode(new HashMap<>())); @@ -125,7 +126,6 @@ class SchedulerHandlerTest { private SynchronousResponse jobResponse; private ConfigurationUpdate configurationUpdate; private JsonSchemaValidator jsonSchemaValidator; - private SpecFetcher specFetcher; private JobPersistence jobPersistence; @BeforeEach @@ -134,7 +134,6 @@ void setup() { jobResponse = mock(SynchronousResponse.class, RETURNS_DEEP_STUBS); configurationUpdate = mock(ConfigurationUpdate.class); jsonSchemaValidator = mock(JsonSchemaValidator.class); - specFetcher = mock(SpecFetcher.class); when(completedJob.getStatus()).thenReturn(JobStatus.SUCCEEDED); when(completedJob.getConfig().getConfigType()).thenReturn(ConfigType.SYNC); when(completedJob.getScope()).thenReturn("sync:123"); @@ -151,7 +150,6 @@ void setup() { synchronousSchedulerClient, configurationUpdate, jsonSchemaValidator, - specFetcher, jobPersistence, jobNotifier, mock(WorkflowServiceStubs.class), @@ -216,12 +214,12 @@ void testCheckSourceConnectionFromUpdate() throws IOException, JsonValidationExc final StandardSourceDefinition sourceDefinition = new StandardSourceDefinition() .withDockerRepository(DESTINATION_DOCKER_REPO) .withDockerImageTag(DESTINATION_DOCKER_TAG) - .withSourceDefinitionId(source.getSourceDefinitionId()); + .withSourceDefinitionId(source.getSourceDefinitionId()) + .withSpec(CONNECTOR_SPECIFICATION); when(configRepository.getStandardSourceDefinition(source.getSourceDefinitionId())) .thenReturn(sourceDefinition); when(configRepository.getSourceConnection(source.getSourceId())).thenReturn(source); when(configurationUpdate.source(source.getSourceId(), source.getName(), sourceUpdate.getConnectionConfiguration())).thenReturn(source); - when(specFetcher.getSpec(sourceDefinition)).thenReturn(CONNECTION_SPECIFICATION); final SourceConnection submittedSource = new SourceConnection() .withSourceDefinitionId(source.getSourceDefinitionId()) .withConfiguration(source.getConfiguration()); @@ -232,7 +230,7 @@ void testCheckSourceConnectionFromUpdate() throws IOException, JsonValidationExc any())).thenReturn(source.getConfiguration()); schedulerHandler.checkSourceConnectionFromSourceIdForUpdate(sourceUpdate); - verify(jsonSchemaValidator).ensure(CONNECTION_SPECIFICATION.getConnectionSpecification(), source.getConfiguration()); + verify(jsonSchemaValidator).ensure(CONNECTOR_SPECIFICATION.getConnectionSpecification(), source.getConfiguration()); verify(synchronousSchedulerClient).createSourceCheckConnectionJob(submittedSource, DESTINATION_DOCKER_IMAGE); } @@ -245,16 +243,15 @@ void testGetSourceSpec() throws JsonValidationException, IOException, ConfigNotF .withName("name") .withDockerRepository(SOURCE_DOCKER_REPO) .withDockerImageTag(SOURCE_DOCKER_TAG) - .withSourceDefinitionId(sourceDefinitionIdRequestBody.getSourceDefinitionId()); + .withSourceDefinitionId(sourceDefinitionIdRequestBody.getSourceDefinitionId()) + .withSpec(CONNECTOR_SPECIFICATION); when(configRepository.getStandardSourceDefinition(sourceDefinitionIdRequestBody.getSourceDefinitionId())) .thenReturn(sourceDefinition); - when(specFetcher.getSpecJobResponse(sourceDefinition)) - .thenReturn(specResponse); - when(specResponse.getOutput()).thenReturn(CONNECTION_SPECIFICATION); - schedulerHandler.getSourceDefinitionSpecification(sourceDefinitionIdRequestBody); + final SourceDefinitionSpecificationRead response = schedulerHandler.getSourceDefinitionSpecification(sourceDefinitionIdRequestBody); verify(configRepository).getStandardSourceDefinition(sourceDefinitionIdRequestBody.getSourceDefinitionId()); + assertEquals(CONNECTOR_SPECIFICATION.getConnectionSpecification(), response.getConnectionSpecification()); } @Test @@ -262,21 +259,19 @@ void testGetDestinationSpec() throws JsonValidationException, IOException, Confi final DestinationDefinitionIdRequestBody destinationDefinitionIdRequestBody = new DestinationDefinitionIdRequestBody().destinationDefinitionId(UUID.randomUUID()); - final SynchronousResponse specResponse = (SynchronousResponse) this.jobResponse; final StandardDestinationDefinition destinationDefinition = new StandardDestinationDefinition() .withName("name") .withDockerRepository(DESTINATION_DOCKER_REPO) .withDockerImageTag(DESTINATION_DOCKER_TAG) - .withDestinationDefinitionId(destinationDefinitionIdRequestBody.getDestinationDefinitionId()); + .withDestinationDefinitionId(destinationDefinitionIdRequestBody.getDestinationDefinitionId()) + .withSpec(CONNECTOR_SPECIFICATION); when(configRepository.getStandardDestinationDefinition(destinationDefinitionIdRequestBody.getDestinationDefinitionId())) .thenReturn(destinationDefinition); - when(specFetcher.getSpecJobResponse(destinationDefinition)) - .thenReturn(specResponse); - when(specResponse.getOutput()).thenReturn(CONNECTION_SPECIFICATION); - schedulerHandler.getDestinationSpecification(destinationDefinitionIdRequestBody); + final DestinationDefinitionSpecificationRead response = schedulerHandler.getDestinationSpecification(destinationDefinitionIdRequestBody); verify(configRepository).getStandardDestinationDefinition(destinationDefinitionIdRequestBody.getDestinationDefinitionId()); + assertEquals(CONNECTOR_SPECIFICATION.getConnectionSpecification(), response.getConnectionSpecification()); } @Test @@ -335,13 +330,13 @@ void testCheckDestinationConnectionFromUpdate() throws IOException, JsonValidati final StandardDestinationDefinition destinationDefinition = new StandardDestinationDefinition() .withDockerRepository(DESTINATION_DOCKER_REPO) .withDockerImageTag(DESTINATION_DOCKER_TAG) - .withDestinationDefinitionId(destination.getDestinationDefinitionId()); + .withDestinationDefinitionId(destination.getDestinationDefinitionId()) + .withSpec(CONNECTOR_SPECIFICATION); when(configRepository.getStandardDestinationDefinition(destination.getDestinationDefinitionId())) .thenReturn(destinationDefinition); when(configRepository.getDestinationConnection(destination.getDestinationId())).thenReturn(destination); when(configurationUpdate.destination(destination.getDestinationId(), destination.getName(), destinationUpdate.getConnectionConfiguration())) .thenReturn(destination); - when(specFetcher.getSpec(destinationDefinition)).thenReturn(CONNECTION_SPECIFICATION); final DestinationConnection submittedDestination = new DestinationConnection() .withDestinationDefinitionId(destination.getDestinationDefinitionId()) .withConfiguration(destination.getConfiguration()); @@ -352,7 +347,7 @@ void testCheckDestinationConnectionFromUpdate() throws IOException, JsonValidati any())).thenReturn(destination.getConfiguration()); schedulerHandler.checkDestinationConnectionFromDestinationIdForUpdate(destinationUpdate); - verify(jsonSchemaValidator).ensure(CONNECTION_SPECIFICATION.getConnectionSpecification(), destination.getConfiguration()); + verify(jsonSchemaValidator).ensure(CONNECTOR_SPECIFICATION.getConnectionSpecification(), destination.getConfiguration()); verify(synchronousSchedulerClient).createDestinationCheckConnectionJob(submittedDestination, DESTINATION_DOCKER_IMAGE); } diff --git a/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceDefinitionsHandlerTest.java b/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceDefinitionsHandlerTest.java index 1c9ae3e9306b..3d1fc8f41f94 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceDefinitionsHandlerTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceDefinitionsHandlerTest.java @@ -26,9 +26,9 @@ import io.airbyte.config.persistence.ConfigNotFoundException; import io.airbyte.config.persistence.ConfigRepository; import io.airbyte.protocol.models.ConnectorSpecification; -import io.airbyte.scheduler.client.CachingSynchronousSchedulerClient; import io.airbyte.scheduler.client.SynchronousJobMetadata; import io.airbyte.scheduler.client.SynchronousResponse; +import io.airbyte.scheduler.client.SynchronousSchedulerClient; import io.airbyte.server.services.AirbyteGithubStore; import io.airbyte.validation.json.JsonValidationException; import java.io.IOException; @@ -48,7 +48,7 @@ class SourceDefinitionsHandlerTest { private StandardSourceDefinition source; private SourceDefinitionsHandler sourceHandler; private Supplier uuidSupplier; - private CachingSynchronousSchedulerClient schedulerSynchronousClient; + private SynchronousSchedulerClient schedulerSynchronousClient; private AirbyteGithubStore githubStore; @SuppressWarnings("unchecked") @@ -56,7 +56,7 @@ class SourceDefinitionsHandlerTest { void setUp() { configRepository = mock(ConfigRepository.class); uuidSupplier = mock(Supplier.class); - schedulerSynchronousClient = spy(CachingSynchronousSchedulerClient.class); + schedulerSynchronousClient = spy(SynchronousSchedulerClient.class); githubStore = mock(AirbyteGithubStore.class); source = generateSource(); @@ -190,7 +190,6 @@ void testUpdateSourceDefinition() throws ConfigNotFoundException, IOException, J assertEquals(newDockerImageTag, sourceDefinitionRead.getDockerImageTag()); verify(schedulerSynchronousClient).createGetSpecJob(newImageName); verify(configRepository).writeStandardSourceDefinition(updatedSource); - verify(schedulerSynchronousClient).resetCache(); } @Nested diff --git a/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceHandlerTest.java b/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceHandlerTest.java index e63074131d28..f5dd4ad80a04 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceHandlerTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceHandlerTest.java @@ -33,7 +33,6 @@ import io.airbyte.config.persistence.split_secrets.JsonSecretsProcessor; import io.airbyte.protocol.models.ConnectorSpecification; import io.airbyte.server.converters.ConfigurationUpdate; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.server.helpers.ConnectionHelpers; import io.airbyte.server.helpers.ConnectorSpecificationHelpers; import io.airbyte.server.helpers.SourceHelpers; @@ -55,7 +54,6 @@ class SourceHandlerTest { private SourceHandler sourceHandler; private JsonSchemaValidator validator; private ConnectionsHandler connectionsHandler; - private SpecFetcher specFetcher; private ConfigurationUpdate configurationUpdate; private Supplier uuidGenerator; private JsonSecretsProcessor secretsProcessor; @@ -68,22 +66,22 @@ void setUp() throws IOException { configRepository = mock(ConfigRepository.class); validator = mock(JsonSchemaValidator.class); connectionsHandler = mock(ConnectionsHandler.class); - specFetcher = mock(SpecFetcher.class); configurationUpdate = mock(ConfigurationUpdate.class); uuidGenerator = mock(Supplier.class); secretsProcessor = mock(JsonSecretsProcessor.class); + connectorSpecification = ConnectorSpecificationHelpers.generateConnectorSpecification(); + standardSourceDefinition = new StandardSourceDefinition() .withSourceDefinitionId(UUID.randomUUID()) .withName("marketo") .withDockerRepository("thebestrepo") .withDockerImageTag("thelatesttag") - .withDocumentationUrl("https://wikipedia.org"); + .withDocumentationUrl("https://wikipedia.org") + .withSpec(connectorSpecification); imageName = DockerUtils.getTaggedImageName(standardSourceDefinition.getDockerRepository(), standardSourceDefinition.getDockerImageTag()); - connectorSpecification = ConnectorSpecificationHelpers.generateConnectorSpecification(); - sourceDefinitionSpecificationRead = new SourceDefinitionSpecificationRead() .sourceDefinitionId(standardSourceDefinition.getSourceDefinitionId()) .connectionSpecification(connectorSpecification.getConnectionSpecification()) @@ -91,8 +89,7 @@ void setUp() throws IOException { sourceConnection = SourceHelpers.generateSource(standardSourceDefinition.getSourceDefinitionId()); - sourceHandler = - new SourceHandler(configRepository, validator, specFetcher, connectionsHandler, uuidGenerator, secretsProcessor, configurationUpdate); + sourceHandler = new SourceHandler(configRepository, validator, connectionsHandler, uuidGenerator, secretsProcessor, configurationUpdate); } @Test @@ -105,7 +102,6 @@ void testCreateSource() throws JsonValidationException, ConfigNotFoundException, when(uuidGenerator.get()).thenReturn(sourceConnection.getSourceId()); when(configRepository.getSourceConnection(sourceConnection.getSourceId())).thenReturn(sourceConnection); - when(specFetcher.getSpec(standardSourceDefinition)).thenReturn(connectorSpecification); when(configRepository.getStandardSourceDefinition(sourceDefinitionSpecificationRead.getSourceDefinitionId())) .thenReturn(standardSourceDefinition); when(secretsProcessor.maskSecrets(sourceCreate.getConnectionConfiguration(), sourceDefinitionSpecificationRead.getConnectionSpecification())) @@ -150,7 +146,6 @@ void testUpdateSource() throws JsonValidationException, ConfigNotFoundException, when(configRepository.getSourceConnection(sourceConnection.getSourceId())) .thenReturn(sourceConnection) .thenReturn(expectedSourceConnection); - when(specFetcher.getSpec(standardSourceDefinition)).thenReturn(connectorSpecification); when(configurationUpdate.source(sourceConnection.getSourceId(), updatedSourceName, newConfiguration)) .thenReturn(expectedSourceConnection); @@ -174,7 +169,6 @@ void testGetSource() throws JsonValidationException, ConfigNotFoundException, IO when(configRepository.getStandardSourceDefinition(sourceDefinitionSpecificationRead.getSourceDefinitionId())) .thenReturn(standardSourceDefinition); when(configRepository.getSourceDefinitionFromSource(sourceConnection.getSourceId())).thenReturn(standardSourceDefinition); - when(specFetcher.getSpec(standardSourceDefinition)).thenReturn(connectorSpecification); when(secretsProcessor.maskSecrets(sourceConnection.getConfiguration(), sourceDefinitionSpecificationRead.getConnectionSpecification())) .thenReturn(sourceConnection.getConfiguration()); @@ -206,7 +200,6 @@ void testListSourcesForWorkspace() throws JsonValidationException, ConfigNotFoun when(configRepository.getStandardSourceDefinition(sourceDefinitionSpecificationRead.getSourceDefinitionId())) .thenReturn(standardSourceDefinition); when(configRepository.getSourceDefinitionFromSource(sourceConnection.getSourceId())).thenReturn(standardSourceDefinition); - when(specFetcher.getSpec(standardSourceDefinition)).thenReturn(connectorSpecification); when(secretsProcessor.maskSecrets(sourceConnection.getConfiguration(), sourceDefinitionSpecificationRead.getConnectionSpecification())) .thenReturn(sourceConnection.getConfiguration()); @@ -225,7 +218,6 @@ void testSearchSources() throws JsonValidationException, ConfigNotFoundException when(configRepository.getStandardSourceDefinition(sourceDefinitionSpecificationRead.getSourceDefinitionId())) .thenReturn(standardSourceDefinition); when(configRepository.getSourceDefinitionFromSource(sourceConnection.getSourceId())).thenReturn(standardSourceDefinition); - when(specFetcher.getSpec(standardSourceDefinition)).thenReturn(connectorSpecification); when(secretsProcessor.maskSecrets(sourceConnection.getConfiguration(), sourceDefinitionSpecificationRead.getConnectionSpecification())) .thenReturn(sourceConnection.getConfiguration()); @@ -261,7 +253,6 @@ void testDeleteSource() throws JsonValidationException, ConfigNotFoundException, when(configRepository.getStandardSourceDefinition(sourceDefinitionSpecificationRead.getSourceDefinitionId())) .thenReturn(standardSourceDefinition); when(configRepository.getSourceDefinitionFromSource(sourceConnection.getSourceId())).thenReturn(standardSourceDefinition); - when(specFetcher.getSpec(standardSourceDefinition)).thenReturn(connectorSpecification); when(connectionsHandler.listConnectionsForWorkspace(workspaceIdRequestBody)).thenReturn(connectionReadList); when(secretsProcessor.maskSecrets(sourceConnection.getConfiguration(), sourceDefinitionSpecificationRead.getConnectionSpecification())) .thenReturn(sourceConnection.getConfiguration()); diff --git a/airbyte-server/src/test/java/io/airbyte/server/migration/RunMigrationTest.java b/airbyte-server/src/test/java/io/airbyte/server/migration/RunMigrationTest.java index 962b4509fec7..48752e9d9cb7 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/migration/RunMigrationTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/migration/RunMigrationTest.java @@ -15,7 +15,6 @@ import com.google.common.io.Resources; import io.airbyte.commons.io.Archives; -import io.airbyte.commons.json.Jsons; import io.airbyte.commons.version.AirbyteVersion; import io.airbyte.config.Configs; import io.airbyte.config.DestinationConnection; @@ -38,11 +37,9 @@ import io.airbyte.db.Database; import io.airbyte.db.instance.test.TestDatabaseProviders; import io.airbyte.migrate.Migrations; -import io.airbyte.protocol.models.ConnectorSpecification; import io.airbyte.scheduler.persistence.DefaultJobPersistence; import io.airbyte.scheduler.persistence.JobPersistence; import io.airbyte.server.RunMigration; -import io.airbyte.server.converters.SpecFetcher; import io.airbyte.validation.json.JsonValidationException; import java.io.File; import java.io.IOException; @@ -69,8 +66,6 @@ public class RunMigrationTest { private static final String TARGET_VERSION = Migrations.MIGRATIONS.get(Migrations.MIGRATIONS.size() - 1).getVersion(); private static final String DEPRECATED_SOURCE_DEFINITION_NOT_BEING_USED = "d2147be5-fa36-4936-977e-f031affa5895"; private static final String DEPRECATED_SOURCE_DEFINITION_BEING_USED = "4eb22946-2a79-4d20-a3e6-effd234613c3"; - private static final ConnectorSpecification MOCK_CONNECTOR_SPEC = new ConnectorSpecification() - .withConnectionSpecification(Jsons.deserialize("{}")); private static PostgreSQLContainer container; private static Database jobDatabase; @@ -159,7 +154,6 @@ private void assertPreMigrationConfigs(final ConfigPersistence configPersistence new NoOpSecretsHydrator(), Optional.of(secretPersistence), Optional.of(secretPersistence)); - configRepository.setSpecFetcher(s -> MOCK_CONNECTOR_SPEC); final Map sourceDefinitionsBeforeMigration = configRepository.listStandardSourceDefinitions().stream() .collect(Collectors.toMap(c -> c.getSourceDefinitionId().toString(), c -> c)); assertTrue(sourceDefinitionsBeforeMigration.containsKey(DEPRECATED_SOURCE_DEFINITION_NOT_BEING_USED)); @@ -178,7 +172,6 @@ private void assertPostMigrationConfigs(final ConfigPersistence configPersistenc new NoOpSecretsHydrator(), Optional.of(secretPersistence), Optional.of(secretPersistence)); - configRepository.setSpecFetcher(s -> MOCK_CONNECTOR_SPEC); final UUID workspaceId = configRepository.listStandardWorkspaces(true).get(0).getWorkspaceId(); // originally the default workspace started with a hardcoded id. the migration in version 0.29.0 // took that id and randomized it. we want to check that the id is now NOT that hardcoded id and @@ -350,13 +343,11 @@ private void runMigration(final ConfigPersistence configPersistence, final JobPe new NoOpSecretsHydrator(), Optional.of(secretPersistence), Optional.of(secretPersistence)); - configRepository.setSpecFetcher(s -> MOCK_CONNECTOR_SPEC); try (final RunMigration runMigration = new RunMigration( jobPersistence, configRepository, new AirbyteVersion(TARGET_VERSION), - YamlSeedConfigPersistence.getDefault(), - mock(SpecFetcher.class))) { + YamlSeedConfigPersistence.getDefault())) { runMigration.run(); } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/22f6c74f-5699-40ff-833c-4a879ea40133.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/22f6c74f-5699-40ff-833c-4a879ea40133.json index 7e7a55eacace..0f41fabe8f3a 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/22f6c74f-5699-40ff-833c-4a879ea40133.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/22f6c74f-5699-40ff-833c-4a879ea40133.json @@ -3,5 +3,10 @@ "name": "BigQuery", "dockerRepository": "airbyte/destination-bigquery", "dockerImageTag": "0.2.0", - "documentationUrl": "https://docs.airbyte.io/integrations/destinations/bigquery" + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/bigquery", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/bigquery", + "connectionSpecification": {}, + "supportsIncremental": true + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/25c5221d-dce2-4163-ade9-739ef790f503.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/25c5221d-dce2-4163-ade9-739ef790f503.json index 2bd62e324b94..3f346d797154 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/25c5221d-dce2-4163-ade9-739ef790f503.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/25c5221d-dce2-4163-ade9-739ef790f503.json @@ -3,5 +3,10 @@ "name": "Postgres", "dockerRepository": "airbyte/destination-postgres", "dockerImageTag": "0.2.0", - "documentationUrl": "https://docs.airbyte.io/integrations/destinations/postgres" + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/postgres", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/postgres", + "connectionSpecification": {}, + "supportsIncremental": true + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/424892c4-daac-4491-b35d-c6688ba547ba.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/424892c4-daac-4491-b35d-c6688ba547ba.json index 8d679bb252c8..f4b70b05aaa3 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/424892c4-daac-4491-b35d-c6688ba547ba.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/424892c4-daac-4491-b35d-c6688ba547ba.json @@ -3,5 +3,10 @@ "name": "Snowflake", "dockerRepository": "airbyte/destination-snowflake", "dockerImageTag": "0.2.0", - "documentationUrl": "https://docs.airbyte.io/integrations/destinations/snowflake" + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/snowflake", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/snowflake", + "connectionSpecification": {}, + "supportsIncremental": true + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/8be1cf83-fde1-477f-a4ad-318d23c9f3c6.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/8be1cf83-fde1-477f-a4ad-318d23c9f3c6.json index 3d1a8ea6f848..9df1daa7600d 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/8be1cf83-fde1-477f-a4ad-318d23c9f3c6.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/8be1cf83-fde1-477f-a4ad-318d23c9f3c6.json @@ -3,5 +3,10 @@ "name": "Local CSV", "dockerRepository": "airbyte/destination-csv", "dockerImageTag": "0.2.0", - "documentationUrl": "https://docs.airbyte.io/integrations/destinations/local-csv" + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/local-csv", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/local-csv", + "connectionSpecification": {}, + "supportsIncremental": true + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/a625d593-bba5-4a1c-a53d-2d246268a816.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/a625d593-bba5-4a1c-a53d-2d246268a816.json index d7ecf5e27e83..10188ba37216 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/a625d593-bba5-4a1c-a53d-2d246268a816.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/a625d593-bba5-4a1c-a53d-2d246268a816.json @@ -3,5 +3,10 @@ "name": "Local JSON", "dockerRepository": "airbyte/destination-local-json", "dockerImageTag": "0.2.0", - "documentationUrl": "https://docs.airbyte.io/integrations/destinations/local-json" + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/local-json", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/local-json", + "connectionSpecification": {}, + "supportsIncremental": true + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/af7c921e-5892-4ff2-b6c1-4a5ab258fb7e.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/af7c921e-5892-4ff2-b6c1-4a5ab258fb7e.json index 21018601a713..db3cc13e318f 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/af7c921e-5892-4ff2-b6c1-4a5ab258fb7e.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/af7c921e-5892-4ff2-b6c1-4a5ab258fb7e.json @@ -3,5 +3,10 @@ "name": "MeiliSearch", "dockerRepository": "airbyte/destination-meilisearch", "dockerImageTag": "0.2.0", - "documentationUrl": "https://docs.airbyte.io/integrations/destinations/meilisearch" + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/meilisearch", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/meilisearch", + "connectionSpecification": {}, + "supportsIncremental": true + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/f7a7d195-377f-cf5b-70a5-be6b819019dc.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/f7a7d195-377f-cf5b-70a5-be6b819019dc.json index f686204fee6b..32304def896d 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/f7a7d195-377f-cf5b-70a5-be6b819019dc.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_DESTINATION_DEFINITION/f7a7d195-377f-cf5b-70a5-be6b819019dc.json @@ -3,5 +3,10 @@ "name": "Redshift", "dockerRepository": "airbyte/destination-redshift", "dockerImageTag": "0.2.0", - "documentationUrl": "https://docs.airbyte.io/integrations/destinations/redshift" + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/redshift", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/redshift", + "connectionSpecification": {}, + "supportsIncremental": true + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/00405b19-9768-4e0c-b1ae-9fc2ee2b2a8c.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/00405b19-9768-4e0c-b1ae-9fc2ee2b2a8c.json index 771a38d386fd..44982b32ccab 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/00405b19-9768-4e0c-b1ae-9fc2ee2b2a8c.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/00405b19-9768-4e0c-b1ae-9fc2ee2b2a8c.json @@ -3,5 +3,9 @@ "name": "Looker", "dockerRepository": "airbyte/source-looker", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-looker" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-looker", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/looker", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/2470e835-feaf-4db6-96f3-70fd645acc77.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/2470e835-feaf-4db6-96f3-70fd645acc77.json index df7ea85dc259..4d8144375228 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/2470e835-feaf-4db6-96f3-70fd645acc77.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/2470e835-feaf-4db6-96f3-70fd645acc77.json @@ -3,5 +3,9 @@ "name": "Salesforce", "dockerRepository": "airbyte/source-salesforce-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-salesforce-singer" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-salesforce-singer", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/salesforce", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/2af123bf-0aaf-4e0d-9784-cb497f23741a.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/2af123bf-0aaf-4e0d-9784-cb497f23741a.json index 4ec108f990c2..1b2292279099 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/2af123bf-0aaf-4e0d-9784-cb497f23741a.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/2af123bf-0aaf-4e0d-9784-cb497f23741a.json @@ -3,5 +3,9 @@ "name": "Appstore", "dockerRepository": "airbyte/source-appstore-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-appstore-singer" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-appstore-singer", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/appstore", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/396e4ca3-8a97-4b85-aa4e-c9d8c2d5f992.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/396e4ca3-8a97-4b85-aa4e-c9d8c2d5f992.json index 0d62416473a5..48b6d163439e 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/396e4ca3-8a97-4b85-aa4e-c9d8c2d5f992.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/396e4ca3-8a97-4b85-aa4e-c9d8c2d5f992.json @@ -3,5 +3,9 @@ "name": "Braintree", "dockerRepository": "airbyte/source-braintree-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-braintree-singer" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-braintree-singer", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/braintree", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/41375467-61ae-4204-8e38-e2b8b7365f23.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/41375467-61ae-4204-8e38-e2b8b7365f23.json index 40eb6405be22..b706fc7d9f1a 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/41375467-61ae-4204-8e38-e2b8b7365f23.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/41375467-61ae-4204-8e38-e2b8b7365f23.json @@ -3,5 +3,9 @@ "name": "Slack", "dockerRepository": "airbyte/source-slack-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/repository/docker/airbyte/source-slack-singer" + "documentationUrl": "https://hub.docker.com/repository/docker/airbyte/source-slack-singer", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/slack", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/435bb9a5-7887-4809-aa58-28c27df0d7ad.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/435bb9a5-7887-4809-aa58-28c27df0d7ad.json index a6b3ef3f1b76..1b9e159ba87c 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/435bb9a5-7887-4809-aa58-28c27df0d7ad.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/435bb9a5-7887-4809-aa58-28c27df0d7ad.json @@ -3,5 +3,9 @@ "name": "MySQL", "dockerRepository": "airbyte/source-mysql", "dockerImageTag": "0.2.0", - "documentationUrl": "https://docs.airbyte.io/integrations/sources/mysql" + "documentationUrl": "https://docs.airbyte.io/integrations/sources/mysql", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/mysql", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/445831eb-78db-4b1f-8f1f-0d96ad8739e2.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/445831eb-78db-4b1f-8f1f-0d96ad8739e2.json index f12b4cecfa99..a73646d3aa07 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/445831eb-78db-4b1f-8f1f-0d96ad8739e2.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/445831eb-78db-4b1f-8f1f-0d96ad8739e2.json @@ -3,5 +3,9 @@ "name": "Drift", "dockerRepository": "airbyte/source-drift", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-drift" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-drift", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/drift", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/4eb22946-2a79-4d20-a3e6-effd234613c3.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/4eb22946-2a79-4d20-a3e6-effd234613c3.json index eb083657a0d0..eb7306ee608f 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/4eb22946-2a79-4d20-a3e6-effd234613c3.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/4eb22946-2a79-4d20-a3e6-effd234613c3.json @@ -3,5 +3,9 @@ "name": "Old connector still being used", "dockerRepository": "airbyte/source-mysql", "dockerImageTag": "0.2.0", - "documentationUrl": "https://docs.airbyte.io/integrations/sources/mysql" + "documentationUrl": "https://docs.airbyte.io/integrations/sources/mysql", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/destinations/mysql", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/57eb1576-8f52-463d-beb6-2e107cdf571d.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/57eb1576-8f52-463d-beb6-2e107cdf571d.json index bf5fbf68969b..e681e0eaac14 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/57eb1576-8f52-463d-beb6-2e107cdf571d.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/57eb1576-8f52-463d-beb6-2e107cdf571d.json @@ -3,5 +3,9 @@ "name": "Hubspot", "dockerRepository": "airbyte/source-hubspot-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://https://docs.airbyte.io/integrations/sources/hubspot" + "documentationUrl": "https://https://docs.airbyte.io/integrations/sources/hubspot", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/hubspot", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/59f1e50a-331f-4f09-b3e8-2e8d4d355f44.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/59f1e50a-331f-4f09-b3e8-2e8d4d355f44.json index 3e3cf22a56d7..fca454d4c181 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/59f1e50a-331f-4f09-b3e8-2e8d4d355f44.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/59f1e50a-331f-4f09-b3e8-2e8d4d355f44.json @@ -3,5 +3,9 @@ "name": "Greenhouse", "dockerRepository": "airbyte/source-greenhouse", "dockerImageTag": "0.2.0", - "documentationUrl": "https://https://docs.airbyte.io/integrations/sources/greenhouse" + "documentationUrl": "https://https://docs.airbyte.io/integrations/sources/greenhouse", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/greenhouse", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/68e63de2-bb83-4c7e-93fa-a8a9051e3993.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/68e63de2-bb83-4c7e-93fa-a8a9051e3993.json index 19c37f45bd7a..93812f6b85b2 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/68e63de2-bb83-4c7e-93fa-a8a9051e3993.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/68e63de2-bb83-4c7e-93fa-a8a9051e3993.json @@ -3,5 +3,9 @@ "name": "Jira", "dockerRepository": "airbyte/source-jira", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-jira" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-jira", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/jira", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/71607ba1-c0ac-4799-8049-7f4b90dd50f7.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/71607ba1-c0ac-4799-8049-7f4b90dd50f7.json index 9239ec3348d6..11ce44b6c97e 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/71607ba1-c0ac-4799-8049-7f4b90dd50f7.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/71607ba1-c0ac-4799-8049-7f4b90dd50f7.json @@ -3,5 +3,9 @@ "name": "Google Sheets", "dockerRepository": "airbyte/source-google-sheets", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/repository/docker/airbyte/source-google-sheets" + "documentationUrl": "https://hub.docker.com/repository/docker/airbyte/source-google-sheets", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/google-sheets", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/778daa7c-feaf-4db6-96f3-70fd645acc77.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/778daa7c-feaf-4db6-96f3-70fd645acc77.json index 70cb8a722c71..d98d85610c7e 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/778daa7c-feaf-4db6-96f3-70fd645acc77.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/778daa7c-feaf-4db6-96f3-70fd645acc77.json @@ -3,5 +3,9 @@ "name": "File", "dockerRepository": "airbyte/source-file", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-file" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-file", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/file", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/859e501d-2b67-471f-91bb-1c801414d28f.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/859e501d-2b67-471f-91bb-1c801414d28f.json index e57ba78e2758..80888b0d66a3 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/859e501d-2b67-471f-91bb-1c801414d28f.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/859e501d-2b67-471f-91bb-1c801414d28f.json @@ -3,5 +3,9 @@ "name": "Mixpanel", "dockerRepository": "airbyte/source-mixpanel-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-mixpanel-singer" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-mixpanel-singer", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/mixpanel", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/9e0556f4-69df-4522-a3fb-03264d36b348.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/9e0556f4-69df-4522-a3fb-03264d36b348.json index 710d90117240..75dec47c07dc 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/9e0556f4-69df-4522-a3fb-03264d36b348.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/9e0556f4-69df-4522-a3fb-03264d36b348.json @@ -3,5 +3,9 @@ "name": "Marketo", "dockerRepository": "airbyte/source-marketo-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-marketo-singer" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-marketo-singer", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/marketo", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/9fed261d-d107-47fd-8c8b-323023db6e20.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/9fed261d-d107-47fd-8c8b-323023db6e20.json index 7134b24eed16..5f76fd492695 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/9fed261d-d107-47fd-8c8b-323023db6e20.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/9fed261d-d107-47fd-8c8b-323023db6e20.json @@ -3,5 +3,9 @@ "name": "Exchange Rates Api", "dockerRepository": "airbyte/source-exchangeratesapi-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/integration-singer-exchangeratesapi_io-source" + "documentationUrl": "https://hub.docker.com/r/airbyte/integration-singer-exchangeratesapi_io-source", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/exchangeratesapi", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/aea2fd0d-377d-465e-86c0-4fdc4f688e51.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/aea2fd0d-377d-465e-86c0-4fdc4f688e51.json index 1bbb2570076e..ab9153772842 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/aea2fd0d-377d-465e-86c0-4fdc4f688e51.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/aea2fd0d-377d-465e-86c0-4fdc4f688e51.json @@ -3,5 +3,9 @@ "name": "Zoom", "dockerRepository": "airbyte/source-zoom-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-zoom-singer" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-zoom-singer", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/zoom", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/b03a9f3e-22a5-11eb-adc1-0242ac120002.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/b03a9f3e-22a5-11eb-adc1-0242ac120002.json index b2017c832209..b08cfcc58259 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/b03a9f3e-22a5-11eb-adc1-0242ac120002.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/b03a9f3e-22a5-11eb-adc1-0242ac120002.json @@ -3,5 +3,9 @@ "name": "Mailchimp", "dockerRepository": "airbyte/source-mailchimp", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-mailchimp" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-mailchimp", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/mailchimp", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/b1892b11-788d-44bd-b9ec-3a436f7b54ce.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/b1892b11-788d-44bd-b9ec-3a436f7b54ce.json index f94e9c4f6f48..82140f27bdef 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/b1892b11-788d-44bd-b9ec-3a436f7b54ce.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/b1892b11-788d-44bd-b9ec-3a436f7b54ce.json @@ -3,5 +3,9 @@ "name": "Shopify", "dockerRepository": "airbyte/source-shopify-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-shopify-singer" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-shopify-singer", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/shopify", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/b5ea17b1-f170-46dc-bc31-cc744ca984c1.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/b5ea17b1-f170-46dc-bc31-cc744ca984c1.json index a6d4d6620c81..91c409ba1ed9 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/b5ea17b1-f170-46dc-bc31-cc744ca984c1.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/b5ea17b1-f170-46dc-bc31-cc744ca984c1.json @@ -3,5 +3,9 @@ "name": "Microsoft SQL Server (MSSQL)", "dockerRepository": "airbyte/source-mssql", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-mssql" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-mssql", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/mssql", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/cd42861b-01fc-4658-a8ab-5d11d0510f01.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/cd42861b-01fc-4658-a8ab-5d11d0510f01.json index c4bd3064a9a4..9c96bcbb5294 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/cd42861b-01fc-4658-a8ab-5d11d0510f01.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/cd42861b-01fc-4658-a8ab-5d11d0510f01.json @@ -3,5 +3,9 @@ "name": "Recurly", "dockerRepository": "airbyte/source-recurly", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-recurly" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-recurly", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/recurly", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/d2147be5-fa36-4936-977e-f031affa5895.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/d2147be5-fa36-4936-977e-f031affa5895.json index 03502ee18e91..cce976b11c97 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/d2147be5-fa36-4936-977e-f031affa5895.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/d2147be5-fa36-4936-977e-f031affa5895.json @@ -3,5 +3,9 @@ "name": "Old Connector", "dockerRepository": "airbyte/source-appstore-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-appstore-singer" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-appstore-singer", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/appstore", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/d29764f8-80d7-4dd7-acbe-1a42005ee5aa.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/d29764f8-80d7-4dd7-acbe-1a42005ee5aa.json index 5bc2bdc6ad62..426c111fece4 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/d29764f8-80d7-4dd7-acbe-1a42005ee5aa.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/d29764f8-80d7-4dd7-acbe-1a42005ee5aa.json @@ -3,5 +3,9 @@ "name": "Zendesk Support", "dockerRepository": "airbyte/source-zendesk-support-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-zendesk-support-singer" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-zendesk-support-singer", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/zendesk-support", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/d8313939-3782-41b0-be29-b3ca20d8dd3a.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/d8313939-3782-41b0-be29-b3ca20d8dd3a.json index 81893900811f..5121a300a770 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/d8313939-3782-41b0-be29-b3ca20d8dd3a.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/d8313939-3782-41b0-be29-b3ca20d8dd3a.json @@ -3,5 +3,9 @@ "name": "Intercom", "dockerRepository": "airbyte/source-intercom-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-intercom-singer" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-intercom-singer", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/intercom", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/decd338e-5647-4c0b-adf4-da0e75f5a750.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/decd338e-5647-4c0b-adf4-da0e75f5a750.json index b994bec5ae3f..79bf62ad40a6 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/decd338e-5647-4c0b-adf4-da0e75f5a750.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/decd338e-5647-4c0b-adf4-da0e75f5a750.json @@ -3,5 +3,9 @@ "name": "Postgres", "dockerRepository": "airbyte/source-postgres", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-postgres" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-postgres", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/postgres", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/e094cb9a-26de-4645-8761-65c0c425d1de.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/e094cb9a-26de-4645-8761-65c0c425d1de.json index 88434c56845d..4e59550a3577 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/e094cb9a-26de-4645-8761-65c0c425d1de.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/e094cb9a-26de-4645-8761-65c0c425d1de.json @@ -3,5 +3,9 @@ "name": "Stripe", "dockerRepository": "airbyte/source-stripe-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/integration-singer-stripe-source" + "documentationUrl": "https://hub.docker.com/r/airbyte/integration-singer-stripe-source", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/stripe", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/e7778cfc-e97c-4458-9ecb-b4f2bba8946c.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/e7778cfc-e97c-4458-9ecb-b4f2bba8946c.json index d8a46e6a2651..f265921c0876 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/e7778cfc-e97c-4458-9ecb-b4f2bba8946c.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/e7778cfc-e97c-4458-9ecb-b4f2bba8946c.json @@ -3,5 +3,9 @@ "name": "Facebook Marketing", "dockerRepository": "airbyte/source-facebook-marketing", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-facebook-marketing" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-facebook-marketing", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/facebook", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/e87ffa8e-a3b5-f69c-9076-6011339de1f6.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/e87ffa8e-a3b5-f69c-9076-6011339de1f6.json index d402393168cd..ad2e7c653474 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/e87ffa8e-a3b5-f69c-9076-6011339de1f6.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/e87ffa8e-a3b5-f69c-9076-6011339de1f6.json @@ -3,5 +3,9 @@ "name": "Redshift", "dockerRepository": "airbyte/source-redshift", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/repository/docker/airbyte/source-redshift" + "documentationUrl": "https://hub.docker.com/repository/docker/airbyte/source-redshift", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/redshift", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/eaf50f04-21dd-4620-913b-2a83f5635227.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/eaf50f04-21dd-4620-913b-2a83f5635227.json index ce4f5fdb31dd..9df0eb9bd85c 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/eaf50f04-21dd-4620-913b-2a83f5635227.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/eaf50f04-21dd-4620-913b-2a83f5635227.json @@ -3,5 +3,9 @@ "name": "Microsoft teams", "dockerRepository": "airbyte/source-microsoft-teams", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-microsoft-teams" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-microsoft-teams", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/microsoft-teams", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/ec4b9503-13cb-48ab-a4ab-6ade4be46567.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/ec4b9503-13cb-48ab-a4ab-6ade4be46567.json index b4dbba4fd0e4..5beabc83f6a6 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/ec4b9503-13cb-48ab-a4ab-6ade4be46567.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/ec4b9503-13cb-48ab-a4ab-6ade4be46567.json @@ -3,5 +3,9 @@ "name": "Freshdesk", "dockerRepository": "airbyte/source-freshdesk", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-freshdesk" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-freshdesk", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/freshdesk", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/ed799e2b-2158-4c66-8da4-b40fe63bc72a.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/ed799e2b-2158-4c66-8da4-b40fe63bc72a.json index 15dd7c90595d..ebe806009ad6 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/ed799e2b-2158-4c66-8da4-b40fe63bc72a.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/ed799e2b-2158-4c66-8da4-b40fe63bc72a.json @@ -3,5 +3,9 @@ "name": "Plaid", "dockerRepository": "airbyte/source-plaid", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-plaid" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-plaid", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/plaid", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/ef69ef6e-aa7f-4af1-a01d-ef775033524e.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/ef69ef6e-aa7f-4af1-a01d-ef775033524e.json index 3667ff4b9af1..2bf8badeebe2 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/ef69ef6e-aa7f-4af1-a01d-ef775033524e.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/ef69ef6e-aa7f-4af1-a01d-ef775033524e.json @@ -3,5 +3,9 @@ "name": "GitHub", "dockerRepository": "airbyte/source-github-singer", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-github-singer" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-github-singer", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/github", + "connectionSpecification": {} + } } diff --git a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/fbb5fbe2-16ad-4cf4-af7d-ff9d9c316c87.json b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/fbb5fbe2-16ad-4cf4-af7d-ff9d9c316c87.json index af5a8a8009b0..84bf72d3da15 100644 --- a/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/fbb5fbe2-16ad-4cf4-af7d-ff9d9c316c87.json +++ b/airbyte-server/src/test/resources/migration/dummy_data/config/STANDARD_SOURCE_DEFINITION/fbb5fbe2-16ad-4cf4-af7d-ff9d9c316c87.json @@ -3,5 +3,9 @@ "name": "Sendgrid", "dockerRepository": "airbyte/source-sendgrid", "dockerImageTag": "0.2.0", - "documentationUrl": "https://hub.docker.com/r/airbyte/source-sendgrid" + "documentationUrl": "https://hub.docker.com/r/airbyte/source-sendgrid", + "spec": { + "documentationUrl": "https://docs.airbyte.io/integrations/sources/sendgrid", + "connectionSpecification": {} + } } diff --git a/docker-compose.yaml b/docker-compose.yaml index 31e304b9d61a..9bfd815040c9 100644 --- a/docker-compose.yaml +++ b/docker-compose.yaml @@ -152,7 +152,6 @@ services: - RESOURCE_MEMORY_REQUEST=${RESOURCE_MEMORY_REQUEST} - RESOURCE_MEMORY_LIMIT=${RESOURCE_MEMORY_LIMIT} - SECRET_PERSISTENCE=${SECRET_PERSISTENCE} - - VERSION_0_32_0_FORCE_UPGRADE=${VERSION_0_32_0_FORCE_UPGRADE} ports: - 8001:8001 volumes: