Skip to content

Commit

Permalink
airbyte-ci: multi arch build
Browse files Browse the repository at this point in the history
  • Loading branch information
alafanechere committed Nov 28, 2023
1 parent d97a399 commit 619d58a
Show file tree
Hide file tree
Showing 12 changed files with 111 additions and 67 deletions.
5 changes: 5 additions & 0 deletions .github/workflows/publish_connectors.yml
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,10 @@ on:
type: string
default: ci-runner-connector-publish-large-dagger-0-6-4
required: true
airbyte-ci-binary-url:
description: "URL to airbyte-ci binary"
required: false
default: https://connectors.airbyte.com/airbyte-ci/releases/ubuntu/latest/airbyte-ci
jobs:
publish_connectors:
name: Publish connectors
Expand Down Expand Up @@ -62,6 +66,7 @@ jobs:
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 }}
subcommand: "connectors ${{ github.event.inputs.connectors-options }} publish ${{ github.event.inputs.publish-options }}"
airbyte_ci_binary_url: ${{ github.event.inputs.airbyte-ci-binary-url }}

set-instatus-incident-on-failure:
name: Create Instatus Incident on Failure
Expand Down
34 changes: 22 additions & 12 deletions airbyte-ci/connectors/pipelines/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -124,17 +124,17 @@ At this point you can run `airbyte-ci` commands.

#### Options

| Option | Default value | Mapped environment variable | Description |
| ------------------------------------------ | ---------------------------------------------------------------------------------------------- | ----------------------------- | ------------------------------------------------------------------------------------------- |
| `--enable-dagger-run/--disable-dagger-run` | `--enable-dagger-run`` | | Disables the Dagger terminal UI. | | |
| `--is-local/--is-ci` | `--is-local` | | Determines the environment in which the CLI runs: local environment or CI environment. |
| `--git-branch` | The checked out git branch name | `CI_GIT_BRANCH` | The git branch on which the pipelines will run. |
| `--git-revision` | The current branch head | `CI_GIT_REVISION` | The commit hash on which the pipelines will run. |
| `--diffed-branch` | `origin/master` | | Branch to which the git diff will happen to detect new or modified files. |
| `--gha-workflow-run-id` | | | GHA CI only - The run id of the GitHub action workflow |
| `--ci-context` | `manual` | | The current CI context: `manual` for manual run, `pull_request`, `nightly_builds`, `master` |
| `--pipeline-start-timestamp` | Current epoch time | `CI_PIPELINE_START_TIMESTAMP` | Start time of the pipeline as epoch time. Used for pipeline run duration computation. |
| `--show-dagger-logs/--hide-dagger-logs` | `--hide-dagger-logs` | | Flag to show or hide the dagger logs. |
| Option | Default value | Mapped environment variable | Description |
| ------------------------------------------ | -------------------------------------------------------------------------------------------------------------------------------- | ----------------------------- | ------------------------------------------------------------------------------------------- |
| `--enable-dagger-run/--disable-dagger-run` | `--enable-dagger-run`` | | Disables the Dagger terminal UI. | | | | |
| `--is-local/--is-ci` | `--is-local` | | Determines the environment in which the CLI runs: local environment or CI environment. |
| `--git-branch` | The checked out git branch name | `CI_GIT_BRANCH` | The git branch on which the pipelines will run. |
| `--git-revision` | The current branch head | `CI_GIT_REVISION` | The commit hash on which the pipelines will run. |
| `--diffed-branch` | `origin/master` | | Branch to which the git diff will happen to detect new or modified files. |
| `--gha-workflow-run-id` | | | GHA CI only - The run id of the GitHub action workflow |
| `--ci-context` | `manual` | | The current CI context: `manual` for manual run, `pull_request`, `nightly_builds`, `master` |
| `--pipeline-start-timestamp` | Current epoch time | `CI_PIPELINE_START_TIMESTAMP` | Start time of the pipeline as epoch time. Used for pipeline run duration computation. |
| `--show-dagger-logs/--hide-dagger-logs` | `--hide-dagger-logs` | | Flag to show or hide the dagger logs. |

### <a id="connectors-command-subgroup"></a>`connectors` command subgroup

Expand Down Expand Up @@ -254,6 +254,9 @@ It's mainly purposed for local use.
Build a single connector:
`airbyte-ci connectors --name=source-pokeapi build`

Build a single connector for multiple architectures:
`airbyte-ci connectors --name=source-pokeapi build --architecture=linux/amd64 --architecture=linux/arm64`

Build multiple connectors:
`airbyte-ci connectors --name=source-pokeapi --name=source-bigquery build`

Expand Down Expand Up @@ -290,11 +293,17 @@ flowchart TD
distTar-->connector
normalization--"if supports normalization"-->connector
load[Load to docker host with :dev tag, current platform]
load[Load to docker host with :dev tag]
spec[Get spec]
connector-->spec--"if success"-->load
```

### Options

| Option | Multiple | Default value | Description |
| --------------------- | -------- | -------------- | ----------------------------------------------------------------- |
| `--architecture`/`-a` | True | Local platform | Defines for which architecture the connector image will be built. |

### <a id="connectors-publish-command"></a>`connectors publish` command
Run a publish pipeline for one or multiple connectors.
It's mainly purposed for CI use to release a connector update.
Expand Down Expand Up @@ -433,6 +442,7 @@ This command runs the Python tests for a airbyte-ci poetry package.
## Changelog
| Version | PR | Description |
| ------- | ---------------------------------------------------------- | --------------------------------------------------------------------------------------------------------- |
| 2.8.0 | [#32816](https://github.com/airbytehq/airbyte/pull/32816) | Add `--architecture` option to connector build. |
| 2.7.0 | [#31930](https://github.com/airbytehq/airbyte/pull/31930) | Merge airbyte-ci-internal into airbyte-ci |
| 2.6.0 | [#31831](https://github.com/airbytehq/airbyte/pull/31831) | Add `airbyte-ci format` commands, remove connector-specific formatting check |
| 2.5.9 | [#32427](https://github.com/airbytehq/airbyte/pull/32427) | Re-enable caching for source-postgres |
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,11 +2,15 @@
# Copyright (c) 2023 Airbyte, Inc., all rights reserved.
#

from typing import List

import asyncclick as click
import dagger
from pipelines.airbyte_ci.connectors.build_image.steps import run_connector_build_pipeline
from pipelines.airbyte_ci.connectors.context import ConnectorContext
from pipelines.airbyte_ci.connectors.pipeline import run_connectors_pipelines
from pipelines.cli.dagger_pipeline_command import DaggerPipelineCommand
from pipelines.consts import BUILD_PLATFORMS, LOCAL_BUILD_PLATFORM


@click.command(cls=DaggerPipelineCommand, help="Build all images for the selected connectors.")
Expand All @@ -17,10 +21,19 @@
default=False,
type=bool,
)
@click.option(
"-a",
"--architecture",
"build_architectures",
help="Architecture for which to build the connector image. If not specified, the image will be built for the local architecture.",
multiple=True,
default=[LOCAL_BUILD_PLATFORM],
type=click.Choice(BUILD_PLATFORMS, case_sensitive=True),
)
@click.pass_context
async def build(ctx: click.Context, use_host_gradle_dist_tar: bool) -> bool:
async def build(ctx: click.Context, use_host_gradle_dist_tar: bool, build_architectures: List[str]) -> bool:
"""Runs a build pipeline for the selected connectors."""

build_platforms = [dagger.Platform(architecture) for architecture in build_architectures]
connectors_contexts = [
ConnectorContext(
pipeline_name=f"Build connector {connector.technical_name}",
Expand Down Expand Up @@ -53,6 +66,7 @@ async def build(ctx: click.Context, use_host_gradle_dist_tar: bool) -> bool:
ctx.obj["concurrency"],
ctx.obj["dagger_logs_path"],
ctx.obj["execute_timeout"],
build_platforms,
)

return True
Original file line number Diff line number Diff line change
Expand Up @@ -5,17 +5,16 @@

from __future__ import annotations

import platform
from typing import List

import anyio
import dagger
from connector_ops.utils import ConnectorLanguage
from pipelines.models.steps import StepResult
from pipelines.airbyte_ci.connectors.build_image.steps import python_connectors
from pipelines.airbyte_ci.connectors.build_image.steps import java_connectors, python_connectors
from pipelines.airbyte_ci.connectors.build_image.steps.common import LoadContainerToLocalDockerHost, StepStatus
from pipelines.consts import LOCAL_BUILD_PLATFORM
from pipelines.airbyte_ci.connectors.build_image.steps import java_connectors
from pipelines.airbyte_ci.connectors.context import ConnectorContext
from pipelines.airbyte_ci.connectors.reports import ConnectorReport
from pipelines.models.steps import StepResult


class NoBuildStepForLanguageError(Exception):
Expand All @@ -29,29 +28,33 @@ class NoBuildStepForLanguageError(Exception):
}


async def run_connector_build(context: ConnectorContext) -> StepResult:
async def run_connector_build(context: ConnectorContext, build_platforms: List[dagger.Platform]) -> StepResult:
"""Run a build pipeline for a single connector."""
if context.connector.language not in LANGUAGE_BUILD_CONNECTOR_MAPPING:
raise NoBuildStepForLanguageError(f"No build step for connector language {context.connector.language}.")
return await LANGUAGE_BUILD_CONNECTOR_MAPPING[context.connector.language](context)
return await LANGUAGE_BUILD_CONNECTOR_MAPPING[context.connector.language](context, build_platforms)


async def run_connector_build_pipeline(context: ConnectorContext, semaphore: anyio.Semaphore) -> ConnectorReport:
async def run_connector_build_pipeline(
context: ConnectorContext, semaphore: anyio.Semaphore, build_platforms: List[dagger.Platform]
) -> ConnectorReport:
"""Run a build pipeline for a single connector.
Args:
context (ConnectorContext): The initialized connector context.
semaphore (anyio.Semaphore): The semaphore to use to limit the number of concurrent builds.
build_platforms (List[dagger.Platform]): The platforms for which to build the connector.
Returns:
ConnectorReport: The reports holding builds results.
"""
step_results = []
async with semaphore:
async with context:
build_result = await run_connector_build(context)
build_result = await run_connector_build(context, build_platforms)
per_platform_built_containers = build_result.output_artifact
step_results.append(build_result)
if context.is_local and build_result.status is StepStatus.SUCCESS:
load_image_result = await LoadContainerToLocalDockerHost(context, LOCAL_BUILD_PLATFORM, build_result.output_artifact).run()
load_image_result = await LoadContainerToLocalDockerHost(context, per_platform_built_containers).run()
step_results.append(load_image_result)
context.report = ConnectorReport(context, step_results, name="BUILD RESULTS")
return context.report
Original file line number Diff line number Diff line change
Expand Up @@ -2,14 +2,15 @@
# Copyright (c) 2023 Airbyte, Inc., all rights reserved.
#

import json
from abc import ABC
from typing import List, Tuple

import docker
from dagger import Container, ExecError, Platform, QueryError
from pipelines.airbyte_ci.connectors.context import ConnectorContext
from pipelines.consts import BUILD_PLATFORMS
from pipelines.helpers.utils import export_container_to_tarball
from pipelines.helpers.utils import export_containers_to_tarball
from pipelines.models.steps import Step, StepResult, StepStatus


Expand Down Expand Up @@ -58,26 +59,34 @@ async def _build_connector(self, platform: Platform, *args) -> Container:
class LoadContainerToLocalDockerHost(Step):
IMAGE_TAG = "dev"

def __init__(self, context: ConnectorContext, platform: Platform, containers: dict[Platform, Container]) -> None:
def __init__(self, context: ConnectorContext, containers: dict[Platform, Container]) -> None:
super().__init__(context)
self.platform = platform
self.container = containers[platform]
self.containers = containers

@property
def title(self):
return f"Load {self.image_name}:{self.IMAGE_TAG} for platform {self.platform} to the local docker host."
return f"Load {self.image_name}:{self.IMAGE_TAG} to the local docker host."

@property
def image_name(self) -> Tuple:
return f"airbyte/{self.context.connector.technical_name}"

async def _run(self) -> StepResult:
_, exported_tarball_path = await export_container_to_tarball(self.context, self.container)
container_variants = list(self.containers.values())
_, exported_tar_path = await export_containers_to_tarball(self.context, container_variants)
client = docker.from_env()
try:
with open(exported_tarball_path, "rb") as tarball_content:
new_image = client.images.load(tarball_content.read())[0]
new_image.tag(self.image_name, tag=self.IMAGE_TAG)
return StepResult(self, StepStatus.SUCCESS)
response = client.api.import_image_from_file(str(exported_tar_path), repository=self.image_name, tag=self.IMAGE_TAG)
try:
image_sha = json.loads(response)["status"]
except KeyError:
return StepResult(
self,
StepStatus.FAILURE,
stderr=f"Failed to load image {self.image_name}:{self.IMAGE_TAG} to your Docker host: {response}",
)
return StepResult(
self, StepStatus.SUCCESS, stdout=f"Loaded image {self.image_name}:{self.IMAGE_TAG} to your Docker host ({image_sha})."
)
except ConnectionError:
return StepResult(self, StepStatus.FAILURE, stderr="The connection to the local docker host failed.")
Original file line number Diff line number Diff line change
Expand Up @@ -2,13 +2,13 @@
# Copyright (c) 2023 Airbyte, Inc., all rights reserved.
#

from typing import List, Optional, Tuple, Union

from dagger import Container, Directory, ExecError, File, Host, Platform, QueryError
from typing import List

from dagger import Container, Directory, File, Platform, QueryError
from pipelines.airbyte_ci.connectors.build_image.steps.common import BuildConnectorImagesBase
from pipelines.airbyte_ci.connectors.context import ConnectorContext
from pipelines.airbyte_ci.steps.gradle import GradleTask
from pipelines.consts import LOCAL_BUILD_PLATFORM
from pipelines.dagger.containers import java
from pipelines.models.steps import StepResult, StepStatus

Expand Down Expand Up @@ -49,21 +49,21 @@ async def _build_connector(self, platform: Platform, dist_tar: File) -> Containe
return await java.with_airbyte_java_connector(self.context, dist_tar, platform)


async def run_connector_build(context: ConnectorContext) -> StepResult:
async def run_connector_build(context: ConnectorContext, build_platforms: List[Platform]) -> StepResult:
"""Create the java connector distribution tar file and build the connector image."""

if context.use_host_gradle_dist_tar and context.is_local:
# Special case: use a local dist tar to speed up local development.
dist_dir = await context.dagger_client.host().directory(dist_tar_directory_path(context), include=["*.tar"])
# Speed things up by only building for the local platform.
return await BuildConnectorImages(context, LOCAL_BUILD_PLATFORM).run(dist_dir)
return await BuildConnectorImages(context, *build_platforms).run(dist_dir)

# Default case: distribution tar is built by the dagger pipeline.
build_connector_tar_result = await BuildConnectorDistributionTar(context).run()
if build_connector_tar_result.status is not StepStatus.SUCCESS:
return build_connector_tar_result
dist_dir = await build_connector_tar_result.output_artifact.directory(dist_tar_directory_path(context))
return await BuildConnectorImages(context).run(dist_dir)
return await BuildConnectorImages(context, *build_platforms).run(dist_dir)


def dist_tar_directory_path(context: ConnectorContext) -> str:
Expand Down
Loading

0 comments on commit 619d58a

Please sign in to comment.