Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
23 commits
Select commit Hold shift + click to select a range
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
58 changes: 57 additions & 1 deletion airflow/providers/airbyte/hooks/airbyte.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,23 @@
# under the License.
from __future__ import annotations

import base64
import json
import time
from typing import Any
from typing import TYPE_CHECKING, Any, TypeVar

import aiohttp
from aiohttp import ClientResponseError
from asgiref.sync import sync_to_async

from airflow.exceptions import AirflowException
from airflow.providers.http.hooks.http import HttpHook

if TYPE_CHECKING:
from airflow.models import Connection

T = TypeVar("T", bound=Any)


class AirbyteHook(HttpHook):
"""
Expand Down Expand Up @@ -50,6 +61,51 @@ def __init__(self, airbyte_conn_id: str = "airbyte_default", api_version: str =
super().__init__(http_conn_id=airbyte_conn_id)
self.api_version: str = api_version

async def get_headers_tenants_from_connection(self) -> tuple[dict[str, Any], str]:
"""Get Headers, tenants from the connection details."""
connection: Connection = await sync_to_async(self.get_connection)(self.http_conn_id)
base_url = connection.host

credentials = f"{connection.login}:{connection.password}"
credentials_base64 = base64.b64encode(credentials.encode("utf-8")).decode("utf-8")

authorized_headers = {
"accept": "application/json",
"content-type": "application/json",
"authorization": f"Basic {credentials_base64}",
}

return authorized_headers, base_url

async def get_job_details(self, job_id: int) -> Any:
"""
Uses Http async call to retrieve metadata for a specific job of an Airbyte Sync.

:param job_id: The ID of an Airbyte Sync Job.
"""
headers, base_url = await self.get_headers_tenants_from_connection()
url = f"{base_url}/api/{self.api_version}/jobs/get"
self.log.info("URL for api request: %s", url)
async with aiohttp.ClientSession(headers=headers) as session:
async with session.post(url=url, data=json.dumps({"id": job_id})) as response:
try:
response.raise_for_status()
return await response.json()
except ClientResponseError as e:
msg = f"{e.status}: {e.message} - {e.request_info}"
raise AirflowException(msg)

async def get_job_status(self, job_id: int) -> str:
"""
Retrieves the status for a specific job of an Airbyte Sync.

:param job_id: The ID of an Airbyte Sync Job.
"""
self.log.info("Getting the status of job run %s.", job_id)
response = await self.get_job_details(job_id=job_id)
job_run_status: str = response["job"]["status"]
return job_run_status

def wait_for_job(self, job_id: str | int, wait_seconds: float = 3, timeout: float | None = 3600) -> None:
"""
Poll a job to check if it finishes.
Expand Down
56 changes: 51 additions & 5 deletions airflow/providers/airbyte/operators/airbyte.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,14 @@
# under the License.
from __future__ import annotations

from typing import TYPE_CHECKING, Sequence
import time
from typing import TYPE_CHECKING, Any, Sequence

from airflow.configuration import conf
from airflow.exceptions import AirflowException
from airflow.models import BaseOperator
from airflow.providers.airbyte.hooks.airbyte import AirbyteHook
from airflow.providers.airbyte.triggers.airbyte import AirbyteSyncTrigger

if TYPE_CHECKING:
from airflow.utils.context import Context
Expand All @@ -40,6 +44,7 @@ class AirbyteTriggerSyncOperator(BaseOperator):
:param asynchronous: Optional. Flag to get job_id after submitting the job to the Airbyte API.
This is useful for submitting long running jobs and
waiting on them asynchronously using the AirbyteJobSensor. Defaults to False.
:param deferrable: Run operator in the deferrable mode.
:param api_version: Optional. Airbyte API version. Defaults to "v1".
:param wait_seconds: Optional. Number of seconds between checks. Only used when ``asynchronous`` is False.
Defaults to 3 seconds.
Expand All @@ -48,12 +53,14 @@ class AirbyteTriggerSyncOperator(BaseOperator):
"""

template_fields: Sequence[str] = ("connection_id",)
ui_color = "#6C51FD"

def __init__(
self,
connection_id: str,
airbyte_conn_id: str = "airbyte_default",
asynchronous: bool = False,
deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False),
api_version: str = "v1",
wait_seconds: float = 3,
timeout: float = 3600,
Expand All @@ -66,23 +73,62 @@ def __init__(
self.api_version = api_version
self.wait_seconds = wait_seconds
self.asynchronous = asynchronous
self.deferrable = deferrable

def execute(self, context: Context) -> None:
"""Create Airbyte Job and wait to finish."""
self.hook = AirbyteHook(airbyte_conn_id=self.airbyte_conn_id, api_version=self.api_version)
job_object = self.hook.submit_sync_connection(connection_id=self.connection_id)
hook = AirbyteHook(airbyte_conn_id=self.airbyte_conn_id, api_version=self.api_version)
job_object = hook.submit_sync_connection(connection_id=self.connection_id)
self.job_id = job_object.json()["job"]["id"]
state = job_object.json()["job"]["status"]
end_time = time.time() + self.timeout

self.log.info("Job %s was submitted to Airbyte Server", self.job_id)
if not self.asynchronous:
self.log.info("Waiting for job %s to complete", self.job_id)
self.hook.wait_for_job(job_id=self.job_id, wait_seconds=self.wait_seconds, timeout=self.timeout)
if self.deferrable:
if state in (hook.RUNNING, hook.PENDING, hook.INCOMPLETE):
self.defer(
timeout=self.execution_timeout,
trigger=AirbyteSyncTrigger(
conn_id=self.airbyte_conn_id,
job_id=self.job_id,
end_time=end_time,
poll_interval=60,
),
method_name="execute_complete",
)
elif state == hook.SUCCEEDED:
self.log.info("Job %s completed successfully", self.job_id)
return
elif state == hook.ERROR:
raise AirflowException(f"Job failed:\n{self.job_id}")
elif state == hook.CANCELLED:
raise AirflowException(f"Job was cancelled:\n{self.job_id}")
else:
raise Exception(f"Encountered unexpected state `{state}` for job_id `{self.job_id}")
else:
hook.wait_for_job(job_id=self.job_id, wait_seconds=self.wait_seconds, timeout=self.timeout)
self.log.info("Job %s completed successfully", self.job_id)

return self.job_id

def execute_complete(self, context: Context, event: Any = None) -> None:
"""
Callback for when the trigger fires - returns immediately.

Relies on trigger to throw an exception, otherwise it assumes execution was
successful.
"""
if event["status"] == "error":
raise AirflowException(event["message"])

self.log.info("%s completed successfully.", self.task_id)
return None

def on_kill(self):
"""Cancel the job if task is cancelled."""
hook = AirbyteHook(airbyte_conn_id=self.airbyte_conn_id)
if self.job_id:
self.log.info("on_kill: cancel the airbyte Job %s", self.job_id)
self.hook.cancel_job(self.job_id)
hook.cancel_job(self.job_id)
5 changes: 5 additions & 0 deletions airflow/providers/airbyte/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,11 @@ sensors:
python-modules:
- airflow.providers.airbyte.sensors.airbyte

triggers:
- integration-name: Airbyte
python-modules:
- airflow.providers.airbyte.triggers.airbyte

connection-types:
- hook-class-name: airflow.providers.airbyte.hooks.airbyte.AirbyteHook
connection-type: airbyte
74 changes: 72 additions & 2 deletions airflow/providers/airbyte/sensors/airbyte.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,14 @@
"""This module contains a Airbyte Job sensor."""
from __future__ import annotations

from typing import TYPE_CHECKING, Sequence
import time
import warnings
from typing import TYPE_CHECKING, Any, Sequence

from airflow.exceptions import AirflowException, AirflowSkipException
from airflow.configuration import conf
from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException
from airflow.providers.airbyte.hooks.airbyte import AirbyteHook
from airflow.providers.airbyte.triggers.airbyte import AirbyteSyncTrigger
from airflow.sensors.base import BaseSensorOperator

if TYPE_CHECKING:
Expand All @@ -34,6 +38,7 @@ class AirbyteJobSensor(BaseSensorOperator):

:param airbyte_job_id: Required. Id of the Airbyte job
:param airbyte_conn_id: Optional. The name of the Airflow connection to get
:param deferrable: Run sensor in the deferrable mode.
connection information for Airbyte. Defaults to "airbyte_default".
:param api_version: Optional. Airbyte API version. Defaults to "v1".
"""
Expand All @@ -45,11 +50,30 @@ def __init__(
self,
*,
airbyte_job_id: int,
deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False),
airbyte_conn_id: str = "airbyte_default",
api_version: str = "v1",
**kwargs,
) -> None:
if deferrable:
if "poke_interval" not in kwargs:
# TODO: Remove once deprecated
if "polling_interval" in kwargs:
kwargs["poke_interval"] = kwargs["polling_interval"]
warnings.warn(
"Argument `poll_interval` is deprecated and will be removed "
"in a future release. Please use `poke_interval` instead.",
AirflowProviderDeprecationWarning,
stacklevel=2,
)
else:
kwargs["poke_interval"] = 5

if "timeout" not in kwargs:
kwargs["timeout"] = 60 * 60 * 24 * 7

super().__init__(**kwargs)
self.deferrable = deferrable
self.airbyte_conn_id = airbyte_conn_id
self.airbyte_job_id = airbyte_job_id
self.api_version = api_version
Expand Down Expand Up @@ -79,3 +103,49 @@ def poke(self, context: Context) -> bool:

self.log.info("Waiting for job %s to complete.", self.airbyte_job_id)
return False

def execute(self, context: Context) -> Any:
"""Submits a job which generates a run_id and gets deferred."""
if not self.deferrable:
super().execute(context)
else:
hook = AirbyteHook(airbyte_conn_id=self.airbyte_conn_id)
job = hook.get_job(job_id=(int(self.airbyte_job_id)))
state = job.json()["job"]["status"]
end_time = time.time() + self.timeout

self.log.info("Airbyte Job Id: Job %s", self.airbyte_job_id)

if state in (hook.RUNNING, hook.PENDING, hook.INCOMPLETE):
self.defer(
timeout=self.execution_timeout,
trigger=AirbyteSyncTrigger(
conn_id=self.airbyte_conn_id,
job_id=self.airbyte_job_id,
end_time=end_time,
poll_interval=60,
),
method_name="execute_complete",
)
elif state == hook.SUCCEEDED:
self.log.info("%s completed successfully.", self.task_id)
return
elif state == hook.ERROR:
raise AirflowException(f"Job failed:\n{job}")
elif state == hook.CANCELLED:
raise AirflowException(f"Job was cancelled:\n{job}")
else:
raise Exception(f"Encountered unexpected state `{state}` for job_id `{self.airbyte_job_id}")

def execute_complete(self, context: Context, event: Any = None) -> None:
"""
Callback for when the trigger fires - returns immediately.

Relies on trigger to throw an exception, otherwise it assumes execution was
successful.
"""
if event["status"] == "error":
raise AirflowException(event["message"])

self.log.info("%s completed successfully.", self.task_id)
return None
16 changes: 16 additions & 0 deletions airflow/providers/airbyte/triggers/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
Loading