1
0
mirror of synced 2025-12-19 18:14:56 -05:00

Move source-declarative-manifest to a standard source, published in step with python cdk (#36501)

This commit is contained in:
Ella Rohm-Ensing
2024-04-05 09:42:10 -07:00
committed by GitHub
parent a750e4697f
commit 51d1353593
22 changed files with 1319 additions and 589 deletions

View File

@@ -28,35 +28,6 @@ concurrency:
cancel-in-progress: false
jobs:
# We are using these runners because they are the same as the one for `publish-command.yml`
# One problem we had using `ubuntu-latest` for example is that the user is not root and some commands would fail in
# `manage.sh` (specifically `apt-get`)
start-publish-docker-image-runner-0:
name: Start Build EC2 Runner 0
runs-on: ubuntu-latest
outputs:
label: ${{ steps.start-ec2-runner.outputs.label }}
ec2-instance-id: ${{ steps.start-ec2-runner.outputs.ec2-instance-id }}
steps:
- name: Checkout Airbyte
uses: actions/checkout@v3
with:
repository: airbytehq/airbyte
ref: master
- name: Check PAT rate limits
run: |
./tools/bin/find_non_rate_limited_PAT \
${{ secrets.GH_PAT_BUILD_RUNNER_OSS }} \
${{ secrets.GH_PAT_BUILD_RUNNER_BACKUP }}
- name: Start AWS Runner
id: start-ec2-runner
uses: ./.github/actions/start-aws-runner
with:
aws-access-key-id: ${{ secrets.SELF_RUNNER_AWS_ACCESS_KEY_ID }}
aws-secret-access-key: ${{ secrets.SELF_RUNNER_AWS_SECRET_ACCESS_KEY }}
github-token: ${{ env.PAT }}
label: ${{ github.run_id }}-publisher
build-cdk:
runs-on: ubuntu-latest
steps:
@@ -127,14 +98,10 @@ jobs:
repository: ${{ github.event.inputs.repo }}
ref: ${{ github.event.inputs.gitref }}
token: ${{ secrets.GH_PAT_MAINTENANCE_OSS }}
- name: "Publish Python CDK: bump version"
- name: "Publish Python CDK: bump Poetry package version"
id: bumpversion
run: |
cd airbyte-cdk/python
# Bump docker image version
# TODO: this should not be necessary, as it's the source-declarative-manifest one which should be published separately
pip install bumpversion
bumpversion ${{ github.event.inputs.release-type }}
# Bump package version
poetry version ${{ github.event.inputs.release-type }}
new_version="$(poetry version -s)"
@@ -143,8 +110,8 @@ jobs:
- name: Commit and Push Changes
uses: stefanzweifel/git-auto-commit-action@v4
with:
file_pattern: airbyte-cdk/python/pyproject.toml airbyte-cdk/python/.bumpversion.cfg airbyte-cdk/python/CHANGELOG.md airbyte-cdk/python/Dockerfile
commit_message: 🤖 Bump ${{ github.event.inputs.release-type }} version of Python CDK
file_pattern: airbyte-cdk/python/pyproject.toml airbyte-cdk/python/CHANGELOG.md
commit_message: 🤖 ${{ github.event.inputs.release-type }} bump Python CDK to version $NEW_VERSION
commit_user_name: Octavia Squidington III
commit_user_email: octavia-squidington-iii@users.noreply.github.com
- name: Post failure to Slack channel dev-connectors-extensibility
@@ -221,34 +188,54 @@ jobs:
env:
SLACK_BOT_TOKEN: ${{ secrets.SLACK_BOT_TOKEN_AIRBYTE_TEAM }}
publish-docker-image:
timeout-minutes: 240
bump-manifest-source:
name: Bump CDK dependency of source-declarative-manifest
needs:
- start-publish-docker-image-runner-0
- bump-version
- publish-cdk
runs-on: runner-pool-${{ github.run_id }}
runs-on: ubuntu-latest
steps:
- uses: actions/setup-python@v4
with:
python-version: "3.10"
- name: Install Poetry
id: install_poetry
uses: snok/install-poetry@v1
- name: Checkout Airbyte
uses: actions/checkout@v3
with:
repository: airbytehq/airbyte
repository: ${{ github.event.inputs.repo }}
ref: ${{ github.event.inputs.gitref }}
- name: Install Python
uses: actions/setup-python@v4
- name: Bump CDK dependency of source-declarative-manifest
run: |
cd airbyte-integrations/connectors/source-declarative-manifest
poetry add airbyte-cdk==${{needs.bump-version.outputs.new_cdk_version}}
- name: Bump version of source-declarative-manifest
uses: ./.github/actions/run-airbyte-ci
with:
python-version: "3.10"
- name: Publish source-declarative-manifest
id: publish
env:
DOCKER_HUB_USERNAME: ${{ secrets.DOCKER_HUB_USERNAME }}
DOCKER_HUB_PASSWORD: ${{ secrets.DOCKER_HUB_PASSWORD }}
# Oracle expects this variable to be set. Although usually present, this is not set by default on Github virtual runners.
TZ: UTC
uses: Wandalen/wretry.action@v1.0.42
context: "master" # TODO: figure out why changing this yells with `The ci_gcs_credentials was not set on this PipelineContext.`
dagger_cloud_token: ${{ secrets.DAGGER_CLOUD_TOKEN_2 }}
docker_hub_password: ${{ secrets.DOCKER_HUB_PASSWORD }}
docker_hub_username: ${{ secrets.DOCKER_HUB_USERNAME }}
gcp_gsm_credentials: ${{ secrets.GCP_GSM_CREDENTIALS }}
gcs_credentials: ${{ secrets.METADATA_SERVICE_PROD_GCS_CREDENTIALS }}
github_token: ${{ secrets.GITHUB_TOKEN }}
metadata_service_gcs_credentials: ${{ secrets.METADATA_SERVICE_PROD_GCS_CREDENTIALS }}
sentry_dsn: ${{ secrets.SENTRY_AIRBYTE_CI_DSN }}
slack_webhook_url: ${{ secrets.PUBLISH_ON_MERGE_SLACK_WEBHOOK }}
spec_cache_gcs_credentials: ${{ secrets.SPEC_CACHE_SERVICE_ACCOUNT_KEY_PUBLISH }}
s3_build_cache_access_key_id: ${{ secrets.SELF_RUNNER_AWS_ACCESS_KEY_ID }}
s3_build_cache_secret_key: ${{ secrets.SELF_RUNNER_AWS_SECRET_ACCESS_KEY }}
# There is no pull request number as we do this manually, so will just reference when we started doing it manually for now
subcommand: "connectors --concurrency=1 --execute-timeout=3600 --name=source-declarative-manifest bump_version ${{ github.event.inputs.release-type }} '36501' 'Bump CDK version to ${{ steps.bumpversion.outputs.NEW_VERSION }}'"
python_registry_token: ${{ secrets.PYPI_TOKEN }}
- name: Commit and Push Changes
uses: stefanzweifel/git-auto-commit-action@v4
with:
command: |
docker login -u ${DOCKER_HUB_USERNAME} -p ${DOCKER_HUB_PASSWORD}
./tools/integrations/manage.sh publish airbyte-cdk/python false
file_pattern: docs/integrations/sources/low-code.md airbyte-integrations/connectors/source-declarative-manifest/*
commit_message: 🤖 Cut version ${{needs.bump-version.outputs.new_cdk_version}} of source-declarative-manifest
commit_user_name: Octavia Squidington III
commit_user_email: octavia-squidington-iii@users.noreply.github.com
- name: Post failure to Slack channel dev-connectors-extensibility
if: ${{ failure() }}
uses: slackapi/slack-github-action@v1.23.0
@@ -257,13 +244,20 @@ jobs:
channel-id: C04J1M66D8B
payload: |
{
"text": "Error during `publish-docker-image` while publishing Python CDK!",
"text": ":warning: A new version of Python CDK has been released but Connector Builder hasn't been automatically updated",
"blocks": [
{
"type": "section",
"text": {
"type": "mrkdwn",
"text": "Error while publishing Docker image following Python CDK release!"
"text": "A new version of Python CDK has been released with <https://github.com/airbytehq/airbyte/blob/master/airbyte-cdk/python/CHANGELOG.md|changelog>: ${{ github.event.inputs.changelog-message }}\n\n"
}
},
{
"type": "section",
"text": {
"type": "mrkdwn",
"text": ":warning: Could not automatically create a PR for Connector Builder>\n"
}
},
{
@@ -382,33 +376,3 @@ jobs:
}
env:
SLACK_BOT_TOKEN: ${{ secrets.SLACK_BOT_TOKEN_AIRBYTE_TEAM }}
# In case of self-hosted EC2 errors, remove this block.
stop-publish-docker-image-runner-0:
if: ${{ always() }} # required to stop the runner even if the error happened in the previous jobs
name: Stop Build EC2 Runner
needs:
- start-publish-docker-image-runner-0 # required to get output from the start-runner job
- publish-docker-image # required to wait when the main job is done
runs-on: ubuntu-latest
steps:
- name: Configure AWS credentials
uses: aws-actions/configure-aws-credentials@v1
with:
aws-access-key-id: ${{ secrets.SELF_RUNNER_AWS_ACCESS_KEY_ID }}
aws-secret-access-key: ${{ secrets.SELF_RUNNER_AWS_SECRET_ACCESS_KEY }}
aws-region: us-east-2
- name: Checkout Airbyte
uses: actions/checkout@v3
- name: Check PAT rate limits
run: |
./tools/bin/find_non_rate_limited_PAT \
${{ secrets.GH_PAT_BUILD_RUNNER_OSS }} \
${{ secrets.GH_PAT_BUILD_RUNNER_BACKUP }}
- name: Stop EC2 runner
uses: airbytehq/ec2-github-runner@base64v1.1.0
with:
mode: stop
github-token: ${{ env.PAT }}
label: ${{ needs.start-publish-docker-image-runner-0.outputs.label }}
ec2-instance-id: ${{ needs.start-publish-docker-image-runner-0.outputs.ec2-instance-id }}

View File

@@ -23,11 +23,8 @@ jobs:
permission: write
commands: |
test
legacy-test
test-performance
publish
publish-java-cdk
legacy-publish
connector-performance
static-args: |
repo=${{ steps.getref.outputs.repo }}

View File

@@ -1,5 +0,0 @@
[bumpversion]
current_version = 0.78.9
commit = False
[bumpversion:file:Dockerfile]

View File

@@ -1,36 +0,0 @@
FROM python:3.9.18-alpine3.18 as base
# build and load all requirements
FROM base as builder
WORKDIR /airbyte/integration_code
# upgrade pip to the latest version
RUN apk --no-cache upgrade \
&& pip install --upgrade pip \
&& apk --no-cache add tzdata build-base
# install airbyte-cdk
RUN pip install --prefix=/install airbyte-cdk==0.78.9
# build a clean environment
FROM base
WORKDIR /airbyte/integration_code
# copy all loaded and built libraries to a pure basic image
COPY --from=builder /install /usr/local
# add default timezone settings
COPY --from=builder /usr/share/zoneinfo/Etc/UTC /etc/localtime
RUN echo "Etc/UTC" > /etc/timezone
# bash is installed for more convenient debugging.
RUN apk --no-cache add bash
# copy payload code only
COPY source_declarative_manifest/main.py ./
ENV AIRBYTE_ENTRYPOINT "python /airbyte/integration_code/main.py"
ENTRYPOINT ["python", "/airbyte/integration_code/main.py"]
# needs to be the same as CDK
LABEL io.airbyte.version=0.78.9
LABEL io.airbyte.name=airbyte/source-declarative-manifest

View File

@@ -1,50 +0,0 @@
# Declarative manifest source
This is a generic source that takes the declarative manifest via a key `__injected_declarative_manifest` of its config.
## Execution
This entrypoint is used for connectors created by the connector builder. These connector's spec is defined in their manifest, which is defined in the config's "__injected_declarative_manifest" field. This allows this entrypoint to be used with any connector manifest.
The spec operation is not supported because the config is not known when running a spec.
## Local development
#### Building
When running a connector locally, you will need to make sure that the CDK generated artifacts are built. Run
```bash
# from airbyte-cdk/python
poetry install
poetry run poe build
```
### Locally running the connector
See `pokeapi_config.json` for an example of a config file that can be passed into the connector.
```bash
# from /airbyte-cdk/python/source-declarative-manifest
poetry run python main.py check --config secrets/config.json
poetry run python main.py discover --config secrets/config.json
poetry run python main.py read --config secrets/config.json --catalog integration_tests/configured_catalog.json
```
### Locally running the connector docker image
#### Build
First, make sure you build the latest Docker image:
```bash
# from airbyte-cdk/python
docker build -t airbyte/source-declarative-manifest:dev .
```
#### Run
Then run any of the connector commands as follows:
```
docker run --rm -v $(pwd)/secrets:/secrets airbyte/source-declarative-manifest:dev check --config /secrets/config.json
docker run --rm -v $(pwd)/secrets:/secrets airbyte/source-declarative-manifest:dev discover --config /secrets/config.json
docker run --rm -v $(pwd)/secrets:/secrets -v $(pwd)/integration_tests:/integration_tests airbyte/source-declarative-manifest:dev read --config /secrets/config.json --catalog /integration_tests/configured_catalog.json
```

View File

@@ -1,29 +0,0 @@
#
# Copyright (c) 2023 Airbyte, Inc., all rights reserved.
#
import sys
from typing import List
from airbyte_cdk.connector import BaseConnector
from airbyte_cdk.entrypoint import AirbyteEntrypoint, launch
from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource
def create_manifest(args: List[str]):
parsed_args = AirbyteEntrypoint.parse_args(args)
if parsed_args.command == "spec":
raise ValueError("spec command is not supported for injected declarative manifest")
config = BaseConnector.read_config(parsed_args.config)
if "__injected_declarative_manifest" not in config:
raise ValueError(
f"Invalid config: `__injected_declarative_manifest` should be provided at the root of the config but config only has keys {list(config.keys())}"
)
return ManifestDeclarativeSource(config.get("__injected_declarative_manifest"))
if __name__ == "__main__":
source = create_manifest(sys.argv[1:])
launch(source, sys.argv[1:])

View File

@@ -0,0 +1,6 @@
# THIS STOPS SOME MODELS TESTS FROM FALLING OVER. IT'S A HACK, WE SHOULD PIN DOWN WHAT'S ACTUALLY GOING ON HERE
# Import the thing that needs to be imported to stop the tests from falling over
from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource
# "Use" the thing so that the linter doesn't complain
placeholder = ManifestDeclarativeSource

View File

@@ -0,0 +1,84 @@
# Declarative-Manifest source connector
This is the repository for the Declarative-Manifest source connector, written in Python.
The declarative manifest source connector is a special connector that can create an arbitrary source
connector from a declarative manifest file. This allows users to create a source connector without writing any code.
**Note**: This connector is managed by the Airbyte Python CDK release process. It can be run as a standalone connector
in Docker and PyAirbyte, but is not yet meant to be run in the platform as a standalone connector. This source is
an interface to the low-code CDK and as such, should not be modified without a corresponding CDK change.
## Local development
### Prerequisites
* Python (~=3.9)
* Poetry (~=1.7) - installation instructions [here](https://python-poetry.org/docs/#installation)
### Installing the connector
From this connector directory, run:
```bash
poetry install
```
### Create credentials
The credentials for source-declarative-manifest are a little different. Your `config` will need to contain the
injected declarative manifest, as indicated in the `spec`. It will also need to contain the fields that the spec
coming out of the manifest requires. An example is available in `integration_tests/pokeapi_config.json`. To use
this example in the following instructions, copy this file to `secrets/config.json`.
### Locally running the connector
```
poetry run source-declarative-manifest spec
poetry run source-declarative-manifest check --config secrets/config.json
poetry run source-declarative-manifest discover --config secrets/config.json
poetry run source-declarative-manifest read --config secrets/config.json --catalog sample_files/configured_catalog.json
```
### Running unit tests
To run unit tests locally, from the connector directory run:
```
poetry run pytest unit_tests
```
### Building the docker image
1. Install [`airbyte-ci`](https://github.com/airbytehq/airbyte/blob/master/airbyte-ci/connectors/pipelines/README.md)
2. Run the following command to build the docker image:
```bash
airbyte-ci connectors --name=source-declarative-manifest build
```
An image will be available on your host with the tag `airbyte/source-declarative-manifest:dev`.
### Running as a docker container
Then run any of the connector commands as follows:
```
docker run --rm airbyte/source-declarative-manifest:dev spec
docker run --rm -v $(pwd)/secrets:/secrets airbyte/source-declarative-manifest:dev check --config /secrets/config.json
docker run --rm -v $(pwd)/secrets:/secrets airbyte/source-declarative-manifest:dev discover --config /secrets/config.json
docker run --rm -v $(pwd)/secrets:/secrets -v $(pwd)/integration_tests:/integration_tests airbyte/source-declarative-manifest:dev read --config /secrets/config.json --catalog /integration_tests/configured_catalog.json
```
### Running our CI test suite
You can run our full test suite locally using [`airbyte-ci`](https://github.com/airbytehq/airbyte/blob/master/airbyte-ci/connectors/pipelines/README.md):
```bash
airbyte-ci connectors --name=source-declarative-manifest test
```
This source does not currently pass the full test suite.
### Dependency Management
The manifest declarative source is built to be an interface to the low-code CDK source. This means that
this source should not have any production dependencies other than the Airbyte Python CDK. If for some reason
you feel that a new dependency is needed, you likely want to add it to the CDK instead. It is expected
that a given version of the source-declarative-manifest connector corresponds to the same version in
its CDK dependency.
## Publishing a new version of the connector
New versions of this connector should only be published (automatically) via the manual Airbyte CDK release process.
If you want to make a change to this connector that is not a result of a CDK change and a corresponding
CDK dependency bump, please reach out to the Connector Extensibility team for guidance.

View File

@@ -0,0 +1,8 @@
#
# Copyright (c) 2023 Airbyte, Inc., all rights reserved.
#
from source_declarative_manifest.run import run
if __name__ == "__main__":
run()

View File

@@ -0,0 +1,33 @@
data:
ab_internal:
ql: 100
sl: 100
connectorBuildOptions:
baseImage: docker.io/airbyte/python-connector-base:1.2.0@sha256:c22a9d97464b69d6ef01898edf3f8612dc11614f05a84984451dde195f337db9
connectorSubtype: api
connectorType: source
definitionId: 64a2f99c-542f-4af8-9a6f-355f1217b436
# This version should not be updated manually - it is updated by the CDK release workflow.
dockerImageTag: 0.78.9
dockerRepository: airbyte/source-declarative-manifest
# This page is hidden from the docs for now, since the connector is not in any Airbyte registries.
documentationUrl: https://docs.airbyte.com/integrations/sources/low-code
githubIssueLabel: source-declarative-manifest
license: MIT
name: Low-Code Source
registries:
# The path for using this source in the Airbyte UI is still with the connector builder for now.
cloud:
enabled: false
oss:
enabled: false
releaseDate: 2023-03-01
releaseStage: alpha
remoteRegistries:
pypi:
enabled: true
packageName: airbyte-source-declarative-manifest
supportLevel: community
tags:
- language:python
metadataSpecVersion: "1.0"

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,28 @@
[build-system]
requires = [ "poetry-core>=1.0.0",]
build-backend = "poetry.core.masonry.api"
[tool.poetry]
version = "0.78.9"
name = "source-declarative-manifest"
description = "Base source implementation for low-code sources."
authors = [ "Airbyte <contact@airbyte.io>",]
license = "MIT"
readme = "README.md"
documentation = "https://docs.airbyte.com/integrations/sources/low-code"
homepage = "https://airbyte.com"
repository = "https://github.com/airbytehq/airbyte"
[[tool.poetry.packages]]
include = "source_declarative_manifest"
[tool.poetry.dependencies]
python = "^3.9,<3.12"
airbyte-cdk = "0.78.9"
[tool.poetry.scripts]
source-declarative-manifest = "source_declarative_manifest.run:run"
[tool.poetry.group.dev.dependencies]
pytest-mock = "^3.6.1"
pytest = "^6.2"
requests-mock = "^1.9.3"

View File

@@ -0,0 +1,53 @@
#
# Copyright (c) 2023 Airbyte, Inc., all rights reserved.
#
import argparse
import json
import pkgutil
import sys
from typing import List
from airbyte_cdk.connector import BaseConnector
from airbyte_cdk.entrypoint import AirbyteEntrypoint, launch
from airbyte_cdk.models import AirbyteMessage, ConnectorSpecification, Type
from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource
def handle_command(args: List[str]) -> None:
"""Overrides the spec command to return the generalized spec for the declarative manifest source.
This is different from a typical low-code, but built and published separately source built as a ManifestDeclarativeSource,
because that will have a spec method that returns the spec for that specific source. Other than spec,
the generalized connector behaves the same as any other, since the manifest is provided in the config.
"""
if args[0] == "spec":
json_spec = pkgutil.get_data("source_declarative_manifest", "spec.json")
spec_obj = json.loads(json_spec)
spec = ConnectorSpecification.parse_obj(spec_obj)
message = AirbyteMessage(type=Type.SPEC, spec=spec)
print(AirbyteEntrypoint.airbyte_message_to_string(message))
else:
source = create_manifest(args)
launch(source, sys.argv[1:])
def create_manifest(args: List[str]) -> ManifestDeclarativeSource:
"""Creates the source with the injected config.
This essentially does what other low-code sources do at build time, but at runtime,
with a user-provided manifest in the config. This better reflects what happens in the
connector builder.
"""
parsed_args = AirbyteEntrypoint.parse_args(args)
config = BaseConnector.read_config(parsed_args.config)
if "__injected_declarative_manifest" not in config:
raise ValueError(
f"Invalid config: `__injected_declarative_manifest` should be provided at the root of the config but config only has keys {list(config.keys())}"
)
return ManifestDeclarativeSource(config.get("__injected_declarative_manifest"))
def run():
args = sys.argv[1:]
handle_command(args)

View File

@@ -0,0 +1,17 @@
{
"documentationUrl": "https://docs.airbyte.com/integrations/sources/low-code",
"connectionSpecification": {
"$schema": "http://json-schema.org/draft-07/schema#",
"title": "Low-code source spec",
"type": "object",
"required": ["__injected_declarative_manifest"],
"additionalProperties": true,
"properties": {
"__injected_declarative_manifest": {
"title": "Low-code manifest",
"type": "object",
"description": "The low-code manifest that defines the components of the source."
}
}
}
}

View File

@@ -7,7 +7,7 @@ import json
import pytest
from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource
from source_declarative_manifest.main import create_manifest
from source_declarative_manifest.run import create_manifest, handle_command
CONFIG = {
"__injected_declarative_manifest": {
@@ -59,9 +59,8 @@ def config_file_without_injection(tmp_path):
return config_file
def test_on_spec_command_then_raise_value_error():
with pytest.raises(ValueError):
create_manifest(["spec"])
def test_spec_does_not_raise_value_error():
handle_command(["spec"])
def test_given_no_injected_declarative_manifest_then_raise_value_error(config_file_without_injection):

View File

@@ -0,0 +1,12 @@
# Low-code
This doc is currently used for changelog purposes only internally, so that we can make sure that the source stays
up-to-date with the latest version of the CDK. The doc is hidden from the docs site.
The changelog below is automatically updated by the `bump_version` command as part of the Airbyte CDK publish flow.
## CHANGELOG
| Version | Date | Pull Request | Subject |
|:--------|:-----------|:---------------------------------------------------------|:---------------------------------------------------------------------|
| 0.78.9 | 2024-04-04 | [00000](https://github.com/airbytehq/airbyte/pull/00000) | Update CDK dependency to version 0.78.9 (before new publishing flow) |

View File

@@ -17,7 +17,8 @@ function getFilenamesInDir(prefix, dir, excludes) {
!(
fileName.endsWith(".inapp.md") ||
fileName.endsWith("-migrations.md") ||
fileName.endsWith(".js")
fileName.endsWith(".js") ||
fileName === "low-code.md"
)
)
.map((fileName) => fileName.replace(".md", ""))

View File

@@ -1,374 +0,0 @@
#!/usr/bin/env bash
set -e
set -x
. tools/lib/lib.sh
# If you are looking at this file because you find yourself needing to publish a connector image manually, you might not need to do all of this!
# If the connector you are publishing is a python connector (e.g. not using our base images), you can do the following:
#
# # NAME="source-foo"; VERSION="1.2.3"
#
# git pull
#
# cd airbyte-integrations/connectors/$NAME
#
# docker buildx build . --platform "linux/amd64,linux/arm64" --tag airbyte/$NAME:latest --push
# docker buildx build . --platform "linux/amd64,linux/arm64" --tag airbyte/$NAME:$VERSION --push
USAGE="
Usage: $(basename "$0") <cmd>
For publish, if you want to push the spec to the spec cache, provide a path to a service account key file that can write to the cache.
Available commands:
scaffold
test <integration_root_path>
build <integration_root_path> [<run_tests>]
publish <integration_root_path> [<run_tests>] [--publish_spec_to_cache] [--publish_spec_to_cache_with_key_file <path to keyfile>] [--pre_release]
publish_external <image_name> <image_version>
"
# these filenames must match DEFAULT_SPEC_FILE and CLOUD_SPEC_FILE in GcsBucketSpecFetcher.java
default_spec_file="spec.json"
cloud_spec_file="spec.cloud.json"
_check_tag_exists() {
DOCKER_CLI_EXPERIMENTAL=enabled docker manifest inspect "$1" > /dev/null
}
_error_if_tag_exists() {
if _check_tag_exists "$1"; then
error "You're trying to push a version that was already released ($1). Make sure you bump it up."
fi
}
cmd_scaffold() {
echo "Scaffolding connector"
(
cd airbyte-integrations/connector-templates/generator &&
./generate.sh "$@"
)
}
cmd_build() {
local path=$1; shift || error "Missing target (root path of integration) $USAGE"
[ -d "$path" ] || error "Path must be the root path of the integration"
local run_tests=$1; shift || run_tests=true
echo "Building $path"
# Note that we are only building (and testing) once on this build machine's architecture
# Learn more @ https://github.com/airbytehq/airbyte/pull/13004
./gradlew --no-daemon --scan "$(_to_gradle_path "$path" clean)"
if [ "$run_tests" = false ] ; then
echo "Building and skipping unit tests + integration tests..."
./gradlew --no-daemon --scan "$(_to_gradle_path "$path" build)" -x check
else
echo "Building and running unit tests + integration tests..."
./gradlew --no-daemon --scan "$(_to_gradle_path "$path" build)"
if test "$path" == "airbyte-integrations/bases/base-normalization"; then
export RANDOM_TEST_SCHEMA="true"
./gradlew --no-daemon --scan :airbyte-integrations:bases:base-normalization:assemble
fi
./gradlew --no-daemon --scan "$(_to_gradle_path "$path" integrationTest)"
fi
}
# Experimental version of the above for a new way to build/tag images
cmd_build_experiment() {
local path=$1; shift || error "Missing target (root path of integration) $USAGE"
[ -d "$path" ] || error "Path must be the root path of the integration"
echo "Building $path"
./gradlew --no-daemon --scan "$(_to_gradle_path "$path" clean)"
./gradlew --no-daemon --scan "$(_to_gradle_path "$path" build)"
# After this happens this image should exist: "image_name:dev"
# Re-tag with CI candidate label
local image_name; image_name=$(_get_docker_image_name "$path/Dockerfile")
local image_version; image_version=$(_get_docker_image_version "$path/Dockerfile")
local image_candidate_tag; image_candidate_tag="$image_version-candidate-$PR_NUMBER"
# If running via the bump-build-test-connector job, re-tag gradle built image following candidate image pattern
if [[ "$GITHUB_JOB" == "bump-build-test-connector" ]]; then
docker tag "$image_name:dev" "$image_name:$image_candidate_tag"
# TODO: docker push "$image_name:$image_candidate_tag"
fi
}
cmd_test() {
local path=$1; shift || error "Missing target (root path of integration) $USAGE"
[ -d "$path" ] || error "Path must be the root path of the integration"
# TODO: needs to know to use alternate image tag from cmd_build_experiment
echo "Running integration tests..."
./gradlew --no-daemon --scan "$(_to_gradle_path "$path" integrationTest)"
}
cmd_publish() {
local path=$1; shift || error "Missing target (root path of integration) $USAGE"
[ -d "$path" ] || error "Path must be the root path of the integration"
local run_tests=$1; shift || run_tests=true
local publish_spec_to_cache
local pre_release
local spec_cache_writer_sa_key_file
while [ $# -ne 0 ]; do
case "$1" in
--publish_spec_to_cache)
publish_spec_to_cache=true
shift 1
;;
--pre_release)
pre_release=true
shift 1
;;
--publish_spec_to_cache_with_key_file)
publish_spec_to_cache=true
spec_cache_writer_sa_key_file="$2"
shift 2
;;
*)
error "Unknown option: $1"
;;
esac
done
if [[ ! $path =~ "connectors" ]]
then
# Do not publish spec to cache in case this is not a connector
publish_spec_to_cache=false
fi
# setting local variables for docker image versioning
local image_name; image_name=$(_get_docker_image_name "$path"/Dockerfile)
local image_version; image_version=$(_get_docker_image_version "$path"/Dockerfile "$pre_release")
local versioned_image=$image_name:$image_version
local latest_image="$image_name" # don't include ":latest", that's assumed here
local build_arch="linux/amd64,linux/arm64"
# learn about this version of Docker
echo "--- docker info ---"
docker --version
docker buildx version
# Install docker emulators
# TODO: Don't run this command on M1 macs locally (it won't work and isn't needed)
apt-get update && apt-get install -y qemu-user-static
# log into docker
if test -z "${DOCKER_HUB_USERNAME}"; then
echo 'DOCKER_HUB_USERNAME not set.';
exit 1;
fi
if test -z "${DOCKER_HUB_PASSWORD}"; then
echo 'DOCKER_HUB_PASSWORD for docker user not set.';
exit 1;
fi
set +x
DOCKER_TOKEN=$(curl -s -H "Content-Type: application/json" -X POST -d '{"username": "'${DOCKER_HUB_USERNAME}'", "password": "'${DOCKER_HUB_PASSWORD}'"}' https://hub.docker.com/v2/users/login/ | jq -r .token)
set -x
echo "image_name $image_name"
echo "versioned_image $versioned_image"
if [ "$pre_release" == "true" ]
then
echo "will skip updating latest_image $latest_image tag due to pre_release"
else
echo "latest_image $latest_image"
fi
# before we start working sanity check that this version has not been published yet, so that we do not spend a lot of
# time building, running tests to realize this version is a duplicate.
_error_if_tag_exists "$versioned_image"
# building the connector
if [ "$path" != "airbyte-cdk/python" ]; then
# The python CDK will already have been built and tested earlier in the github workflow.
cmd_build "$path" "$run_tests"
fi
# in case curing the build / tests someone this version has been published.
_error_if_tag_exists "$versioned_image"
if [[ "airbyte/normalization" == "${image_name}" ]]; then
echo "Publishing normalization images (version: $versioned_image)"
GIT_REVISION=$(git rev-parse HEAD)
# We use a buildx docker container when building multi-stage builds from one docker compose file
# This works because all the images depend only on already public images
docker buildx create --name connector-buildx --driver docker-container --use
# Note: "buildx bake" needs to be run within the directory
local original_pwd=$PWD
cd airbyte-integrations/bases/base-normalization
VERSION=$image_version GIT_REVISION=$GIT_REVISION docker buildx bake \
--set "*.platform=$build_arch" \
-f docker-compose.build.yaml \
--push
if [ "$pre_release" != "true" ]; then
VERSION=latest GIT_REVISION=$GIT_REVISION docker buildx bake \
--set "*.platform=$build_arch" \
-f docker-compose.build.yaml \
--push
fi
docker buildx rm connector-buildx
cd $original_pwd
else
# We have to go arch-by-arch locally (see https://github.com/docker/buildx/issues/59 for more info) due to our base images (e.g. airbyte-integrations/bases/base-java)
# Alternative local approach @ https://github.com/docker/buildx/issues/301#issuecomment-755164475
# We need to use the regular docker buildx driver (not docker container) because we need this intermediate contaiers to be available for later build steps
echo Installing arm64 docker emulation
docker run --privileged --rm tonistiigi/binfmt --install arm64
for arch in $(echo $build_arch | sed "s/,/ /g")
do
# These images aren't needed for the CDK
if [ "$path" != "airbyte-cdk/python" ]; then
echo "building base images for $arch"
docker buildx build -t airbyte/integration-base-java:dev --platform $arch --load airbyte-integrations/bases/base-java
docker buildx build -t airbyte/integration-base:dev --platform $arch --load airbyte-integrations/bases/base
fi
# For a short while (https://github.com/airbytehq/airbyte/pull/25034), destinations rely on the normalization image to build
# Thanks to gradle, destinstaions which need normalization will already have built base-normalization's "build" artifacts
if [[ "$image_name" == *"destination-"* ]]; then
if [ -f "airbyte-integrations/bases/base-normalization/build/sshtunneling.sh" ]; then
docker buildx build -t airbyte/normalization:dev --platform $arch --load airbyte-integrations/bases/base-normalization
fi
fi
local arch_versioned_image=$image_name:`echo $arch | sed "s/\//-/g"`-$image_version
echo "Publishing new version ($arch_versioned_image) from $path"
docker buildx build -t $arch_versioned_image --platform $arch --push $path
docker manifest create $versioned_image --amend $arch_versioned_image
if [ "$pre_release" != "true" ]; then
docker manifest create $latest_image --amend $arch_versioned_image
fi
done
docker manifest push $versioned_image
docker manifest rm $versioned_image
if [ "$pre_release" != "true" ]; then
docker manifest push $latest_image
docker manifest rm $latest_image
fi
# delete the temporary image tags made with arch_versioned_image
sleep 10
for arch in $(echo $build_arch | sed "s/,/ /g")
do
local arch_versioned_tag=`echo $arch | sed "s/\//-/g"`-$image_version
echo "deleting temporary tag: ${image_name}/tags/${arch_versioned_tag}"
TAG_URL="https://hub.docker.com/v2/repositories/${image_name}/tags/${arch_versioned_tag}/" # trailing slash is needed!
set +x
curl -X DELETE -H "Authorization: JWT ${DOCKER_TOKEN}" "$TAG_URL"
set -x
done
fi
# Checking if the image was successfully registered on DockerHub
# see the description of this PR to understand why this is needed https://github.com/airbytehq/airbyte/pull/11654/
sleep 5
# To work for private repos we need a token as well
TAG_URL="https://hub.docker.com/v2/repositories/${image_name}/tags/${image_version}"
set +x
DOCKERHUB_RESPONSE_CODE=$(curl --silent --output /dev/null --write-out "%{http_code}" -H "Authorization: JWT ${DOCKER_TOKEN}" ${TAG_URL})
set -x
if [[ "${DOCKERHUB_RESPONSE_CODE}" == "404" ]]; then
echo "Tag ${image_version} was not registered on DockerHub for image ${image_name}, please try to bump the version again." && exit 1
fi
if [[ "true" == "${publish_spec_to_cache}" ]]; then
echo "Publishing and writing to spec cache."
# use service account key file is provided.
if [[ -n "${spec_cache_writer_sa_key_file}" ]]; then
echo "Using provided service account key"
gcloud auth activate-service-account --key-file "$spec_cache_writer_sa_key_file"
else
echo "Using environment gcloud"
fi
publish_spec_files "$image_name" "$image_version"
else
echo "Publishing without writing to spec cache."
fi
}
cmd_publish_external() {
local image_name=$1; shift || error "Missing target (image name) $USAGE"
# Get version from the command
local image_version=$1; shift || error "Missing target (image version) $USAGE"
echo "image $image_name:$image_version"
echo "Publishing and writing to spec cache."
echo "Using environment gcloud"
publish_spec_files "$image_name" "$image_version"
}
generate_spec_file() {
local image_name=$1; shift || error "Missing target (image name)"
local image_version=$1; shift || error "Missing target (image version)"
local tmp_spec_file=$1; shift || error "Missing target (temp spec file name)"
local deployment_mode=$1; shift || error "Missing target (deployment mode)"
docker run --env DEPLOYMENT_MODE="$deployment_mode" --rm "$image_name:$image_version" spec | \
# 1. filter out any lines that are not valid json.
jq -R "fromjson? | ." | \
# 2. grab any json that has a spec in it.
# 3. if there are more than one, take the first one.
# 4. if there are none, throw an error.
jq -s "map(select(.spec != null)) | map(.spec) | first | if . != null then . else error(\"no spec found\") end" \
> "$tmp_spec_file"
}
publish_spec_files() {
local image_name=$1; shift || error "Missing target (image name)"
local image_version=$1; shift || error "Missing target (image version)"
# publish spec to cache. do so, by running get spec locally and then pushing it to gcs.
local tmp_default_spec_file; tmp_default_spec_file=$(mktemp)
local tmp_cloud_spec_file; tmp_cloud_spec_file=$(mktemp)
# generate oss and cloud spec files
generate_spec_file "$image_name" "$image_version" "$tmp_default_spec_file" "OSS"
generate_spec_file "$image_name" "$image_version" "$tmp_cloud_spec_file" "CLOUD"
gsutil cp "$tmp_default_spec_file" "gs://io-airbyte-cloud-spec-cache/specs/$image_name/$image_version/$default_spec_file"
if cmp --silent -- "$tmp_default_spec_file" "$tmp_cloud_spec_file"; then
echo "This connector has the same spec file for OSS and cloud"
else
echo "Uploading cloud specific spec file"
gsutil cp "$tmp_cloud_spec_file" "gs://io-airbyte-cloud-spec-cache/specs/$image_name/$image_version/$cloud_spec_file"
fi
}
main() {
assert_root
local cmd=$1; shift || error "Missing cmd $USAGE"
cmd_"$cmd" "$@"
}
main "$@"