From 851564fd9f91fc869a50919775914488f68a383b Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 25 Sep 2024 13:26:24 +0530 Subject: [PATCH 01/49] Draft: dbt compile task --- cosmos/airflow/graph.py | 19 ++++++- cosmos/constants.py | 1 + cosmos/operators/airflow_async.py | 91 +++++++++++++++++++++++++++++++ cosmos/operators/base.py | 9 +++ cosmos/operators/local.py | 62 ++++++++++++++++++++- cosmos/settings.py | 4 ++ 6 files changed, 181 insertions(+), 5 deletions(-) create mode 100644 cosmos/operators/airflow_async.py diff --git a/cosmos/airflow/graph.py b/cosmos/airflow/graph.py index 17ee22c95..230c51d63 100644 --- a/cosmos/airflow/graph.py +++ b/cosmos/airflow/graph.py @@ -20,6 +20,7 @@ from cosmos.core.graph.entities import Task as TaskMetadata from cosmos.dbt.graph import DbtNode from cosmos.log import get_logger +from cosmos.settings import dbt_compile_task_id logger = get_logger(__name__) @@ -332,11 +333,22 @@ def build_airflow_graph( for leaf_node_id in leaves_ids: tasks_map[leaf_node_id] >> test_task - create_airflow_task_dependencies(nodes, tasks_map) + if execution_mode == ExecutionMode.AIRFLOW_ASYNC: + compile_task_metadata = TaskMetadata( + id=dbt_compile_task_id, + owner="", # Set appropriate owner if needed + operator_class=f"cosmos.operators.airflow_async.DbtCompileAirflowAsyncOperator", + arguments=task_args, + extra_context={}, + ) + compile_airflow_task = create_airflow_task(compile_task_metadata, dag, task_group=None) + tasks_map[dbt_compile_task_id] = compile_airflow_task + + create_airflow_task_dependencies(nodes, tasks_map, execution_mode) def create_airflow_task_dependencies( - nodes: dict[str, DbtNode], tasks_map: dict[str, Union[TaskGroup, BaseOperator]] + nodes: dict[str, DbtNode], tasks_map: dict[str, Union[TaskGroup, BaseOperator]], execution_mode: ExecutionMode ) -> None: """ Create the Airflow task dependencies between non-test nodes. @@ -344,6 +356,9 @@ def create_airflow_task_dependencies( :param tasks_map: Dictionary mapping dbt nodes (node.unique_id to Airflow task) """ for node_id, node in nodes.items(): + if not node.depends_on and execution_mode == ExecutionMode.AIRFLOW_ASYNC: + tasks_map[dbt_compile_task_id] >> tasks_map[node_id] + for parent_node_id in node.depends_on: # depending on the node type, it will not have mapped 1:1 to tasks_map if (node_id in tasks_map) and (parent_node_id in tasks_map): diff --git a/cosmos/constants.py b/cosmos/constants.py index e9d1aaa6b..64fd42701 100644 --- a/cosmos/constants.py +++ b/cosmos/constants.py @@ -86,6 +86,7 @@ class ExecutionMode(Enum): """ LOCAL = "local" + AIRFLOW_ASYNC = "airflow_async" DOCKER = "docker" KUBERNETES = "kubernetes" AWS_EKS = "aws_eks" diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py new file mode 100644 index 000000000..b02a54416 --- /dev/null +++ b/cosmos/operators/airflow_async.py @@ -0,0 +1,91 @@ +from typing import Any + +from airflow.utils.context import Context + +from cosmos.operators.base import DbtCompileMixin +from cosmos.operators.local import ( + DbtBuildLocalOperator, + DbtDepsLocalOperator, + DbtDocsAzureStorageLocalOperator, + DbtDocsCloudLocalOperator, + DbtDocsGCSLocalOperator, + DbtDocsLocalOperator, + DbtDocsS3LocalOperator, + DbtLocalBaseOperator, + DbtLSLocalOperator, + DbtRunLocalOperator, + DbtRunOperationLocalOperator, + DbtSeedLocalOperator, + DbtSnapshotLocalOperator, + DbtSourceLocalOperator, + DbtTestLocalOperator, +) + + +class DbtBuildAirflowAsyncOperator(DbtBuildLocalOperator): + pass + + +class DbtLSAirflowAsyncOperator(DbtLSLocalOperator): + pass + + +class DbtSeedAirflowAsyncOperator(DbtSeedLocalOperator): + pass + + +class DbtSnapshotAirflowAsyncOperator(DbtSnapshotLocalOperator): + pass + + +class DbtSourceAirflowAsyncOperator(DbtSourceLocalOperator): + pass + + +class DbtRunAirflowAsyncOperator(DbtRunLocalOperator): + pass + + +class DbtTestAirflowAsyncOperator(DbtTestLocalOperator): + pass + + +class DbtRunOperationAirflowAsyncOperator(DbtRunOperationLocalOperator): + pass + + +class DbtDocsAirflowAsyncOperator(DbtDocsLocalOperator): + pass + + +class DbtDocsCloudAirflowAsyncOperator(DbtDocsCloudLocalOperator): + pass + + +class DbtDocsS3AirflowAsyncOperator(DbtDocsS3LocalOperator): + pass + + +class DbtDocsAzureStorageAirflowAsyncOperator(DbtDocsAzureStorageLocalOperator): + pass + + +class DbtDocsGCSAirflowAsyncOperator(DbtDocsGCSLocalOperator): + pass + + +class DbtDepsAirflowAsyncOperator(DbtDepsLocalOperator): + pass + + +class DbtCompileAirflowAsyncOperator(DbtCompileMixin, DbtLocalBaseOperator): + """ + Executes a dbt core build command. + """ + + def __init__(self, *args: Any, **kwargs: Any) -> None: + kwargs["should_upload_compiled_sql"] = True + super().__init__(*args, **kwargs) + + def execute(self, context: Context) -> None: + self.build_and_run_cmd(context=context, cmd_flags=self.add_cmd_flags()) diff --git a/cosmos/operators/base.py b/cosmos/operators/base.py index d82083a23..ed7969ebd 100644 --- a/cosmos/operators/base.py +++ b/cosmos/operators/base.py @@ -429,3 +429,12 @@ def add_cmd_flags(self) -> list[str]: flags.append("--args") flags.append(yaml.dump(self.args)) return flags + + +class DbtCompileMixin: + """ + Mixin for dbt compile command. + """ + + base_cmd = ["compile"] + ui_color = "#877c7c" diff --git a/cosmos/operators/local.py b/cosmos/operators/local.py index 701552f56..3254c8959 100644 --- a/cosmos/operators/local.py +++ b/cosmos/operators/local.py @@ -16,6 +16,7 @@ from airflow.models.taskinstance import TaskInstance from airflow.utils.context import Context from airflow.utils.session import NEW_SESSION, create_session, provide_session +from airflow.version import version as airflow_version from attr import define from cosmos import cache @@ -24,10 +25,10 @@ _get_latest_cached_package_lockfile, is_cache_package_lockfile_enabled, ) -from cosmos.constants import InvocationMode +from cosmos.constants import FILE_SCHEME_AIRFLOW_DEFAULT_CONN_ID_MAP, InvocationMode from cosmos.dbt.project import get_partial_parse_path, has_non_empty_dependencies_file -from cosmos.exceptions import AirflowCompatibilityError -from cosmos.settings import LINEAGE_NAMESPACE +from cosmos.exceptions import AirflowCompatibilityError, CosmosValueError +from cosmos.settings import AIRFLOW_IO_AVAILABLE, LINEAGE_NAMESPACE, remote_target_path, remote_target_path_conn_id try: from airflow.datasets import Dataset @@ -131,6 +132,7 @@ def __init__( install_deps: bool = False, callback: Callable[[str], None] | None = None, should_store_compiled_sql: bool = True, + should_upload_compiled_sql: bool = False, append_env: bool = True, **kwargs: Any, ) -> None: @@ -139,6 +141,7 @@ def __init__( self.compiled_sql = "" self.freshness = "" self.should_store_compiled_sql = should_store_compiled_sql + self.should_upload_compiled_sql = should_upload_compiled_sql self.openlineage_events_completes: list[RunEvent] = [] self.invocation_mode = invocation_mode self.invoke_dbt: Callable[..., FullOutputSubprocessResult | dbtRunnerResult] @@ -252,6 +255,58 @@ def store_compiled_sql(self, tmp_project_dir: str, context: Context, session: Se else: self.log.info("Warning: ti is of type TaskInstancePydantic. Cannot update template_fields.") + @staticmethod + def _configure_remote_target_path() -> Path | None: + """Configure the remote target path if it is provided.""" + if not remote_target_path: + return None + + _configured_target_path = None + + target_path_str = str(remote_target_path) + + remote_conn_id = remote_target_path_conn_id + if not remote_conn_id: + target_path_schema = target_path_str.split("://")[0] + remote_conn_id = FILE_SCHEME_AIRFLOW_DEFAULT_CONN_ID_MAP.get(target_path_schema, None) # type: ignore[assignment] + if remote_conn_id is None: + return _configured_target_path + + if not AIRFLOW_IO_AVAILABLE: + raise CosmosValueError( + f"You're trying to specify remote target path {target_path_str}, but the required " + f"Object Storage feature is unavailable in Airflow version {airflow_version}. Please upgrade to " + "Airflow 2.8 or later." + ) + + from airflow.io.path import ObjectStoragePath + + _configured_target_path = ObjectStoragePath(target_path_str, conn_id=remote_conn_id) + + if not _configured_target_path.exists(): # type: ignore[no-untyped-call] + _configured_target_path.mkdir(parents=True, exist_ok=True) + + return _configured_target_path + + def upload_compiled_sql(self, tmp_project_dir: str, context: Context) -> None: + """ + Uploads the compiled SQL files from the dbt compile output to the remote store. + """ + if not self.should_upload_compiled_sql: + return + + dest_target_dir = self._configure_remote_target_path() + if not dest_target_dir: + raise CosmosValueError( + "You're trying to upload compiled SQL files, but the remote target path is not configured. " + ) + + from airflow.io.path import ObjectStoragePath + + source_target_dir = ObjectStoragePath(Path(tmp_project_dir) / "target" / "compiled") + + source_target_dir.copy(dest_target_dir, recursive=True) # type: ignore[arg-type] + @provide_session def store_freshness_json(self, tmp_project_dir: str, context: Context, session: Session = NEW_SESSION) -> None: """ @@ -397,6 +452,7 @@ def run_command( self.store_freshness_json(tmp_project_dir, context) self.store_compiled_sql(tmp_project_dir, context) + self.upload_compiled_sql(tmp_project_dir, context) self.handle_exception(result) if self.callback: self.callback(tmp_project_dir) diff --git a/cosmos/settings.py b/cosmos/settings.py index 43abc8897..57e8aeaaa 100644 --- a/cosmos/settings.py +++ b/cosmos/settings.py @@ -34,6 +34,10 @@ remote_cache_dir = conf.get("cosmos", "remote_cache_dir", fallback=None) remote_cache_dir_conn_id = conf.get("cosmos", "remote_cache_dir_conn_id", fallback=None) +dbt_compile_task_id = conf.get("cosmos", "dbt_compile_task_id", fallback="dbt_compile") +remote_target_path = conf.get("cosmos", "remote_target_path", fallback=None) +remote_target_path_conn_id = conf.get("cosmos", "remote_target_path_conn_id", fallback=None) + try: LINEAGE_NAMESPACE = conf.get("openlineage", "namespace") except airflow.exceptions.AirflowConfigException: From 9dc2c9cd4ab77267207e0e62c32bbf13692428c2 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Sun, 29 Sep 2024 19:58:51 +0530 Subject: [PATCH 02/49] Put compiled files under dag_id folder & refactor few snippets --- cosmos/airflow/graph.py | 43 ++++++++++++++++++++----------- cosmos/operators/airflow_async.py | 5 ---- cosmos/operators/local.py | 23 ++++++++++++----- 3 files changed, 44 insertions(+), 27 deletions(-) diff --git a/cosmos/airflow/graph.py b/cosmos/airflow/graph.py index 230c51d63..d84a1fafb 100644 --- a/cosmos/airflow/graph.py +++ b/cosmos/airflow/graph.py @@ -253,6 +253,30 @@ def generate_task_or_group( return task_or_group +def _add_dbt_compile_task( + nodes: dict[str, DbtNode], + dag: DAG, + execution_mode: ExecutionMode, + task_args: dict[str, Any], + tasks_map: dict[str, Any], +) -> None: + if execution_mode != ExecutionMode.AIRFLOW_ASYNC: + return + + compile_task_metadata = TaskMetadata( + id=dbt_compile_task_id, + operator_class=f"cosmos.operators.airflow_async.DbtCompileAirflowAsyncOperator", + arguments=task_args, + extra_context={}, + ) + compile_airflow_task = create_airflow_task(compile_task_metadata, dag, task_group=None) + tasks_map[dbt_compile_task_id] = compile_airflow_task + + for node_id, node in nodes.items(): + if not node.depends_on and node_id in tasks_map: + tasks_map[dbt_compile_task_id] >> tasks_map[node_id] + + def build_airflow_graph( nodes: dict[str, DbtNode], dag: DAG, # Airflow-specific - parent DAG where to associate tasks and (optional) task groups @@ -333,22 +357,14 @@ def build_airflow_graph( for leaf_node_id in leaves_ids: tasks_map[leaf_node_id] >> test_task - if execution_mode == ExecutionMode.AIRFLOW_ASYNC: - compile_task_metadata = TaskMetadata( - id=dbt_compile_task_id, - owner="", # Set appropriate owner if needed - operator_class=f"cosmos.operators.airflow_async.DbtCompileAirflowAsyncOperator", - arguments=task_args, - extra_context={}, - ) - compile_airflow_task = create_airflow_task(compile_task_metadata, dag, task_group=None) - tasks_map[dbt_compile_task_id] = compile_airflow_task + _add_dbt_compile_task(nodes, dag, execution_mode, task_args, tasks_map) - create_airflow_task_dependencies(nodes, tasks_map, execution_mode) + create_airflow_task_dependencies(nodes, tasks_map) def create_airflow_task_dependencies( - nodes: dict[str, DbtNode], tasks_map: dict[str, Union[TaskGroup, BaseOperator]], execution_mode: ExecutionMode + nodes: dict[str, DbtNode], + tasks_map: dict[str, Union[TaskGroup, BaseOperator]], ) -> None: """ Create the Airflow task dependencies between non-test nodes. @@ -356,9 +372,6 @@ def create_airflow_task_dependencies( :param tasks_map: Dictionary mapping dbt nodes (node.unique_id to Airflow task) """ for node_id, node in nodes.items(): - if not node.depends_on and execution_mode == ExecutionMode.AIRFLOW_ASYNC: - tasks_map[dbt_compile_task_id] >> tasks_map[node_id] - for parent_node_id in node.depends_on: # depending on the node type, it will not have mapped 1:1 to tasks_map if (node_id in tasks_map) and (parent_node_id in tasks_map): diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index b02a54416..334e074e5 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -1,7 +1,5 @@ from typing import Any -from airflow.utils.context import Context - from cosmos.operators.base import DbtCompileMixin from cosmos.operators.local import ( DbtBuildLocalOperator, @@ -86,6 +84,3 @@ class DbtCompileAirflowAsyncOperator(DbtCompileMixin, DbtLocalBaseOperator): def __init__(self, *args: Any, **kwargs: Any) -> None: kwargs["should_upload_compiled_sql"] = True super().__init__(*args, **kwargs) - - def execute(self, context: Context) -> None: - self.build_and_run_cmd(context=context, cmd_flags=self.add_cmd_flags()) diff --git a/cosmos/operators/local.py b/cosmos/operators/local.py index 3254c8959..25b7d9dde 100644 --- a/cosmos/operators/local.py +++ b/cosmos/operators/local.py @@ -256,10 +256,10 @@ def store_compiled_sql(self, tmp_project_dir: str, context: Context, session: Se self.log.info("Warning: ti is of type TaskInstancePydantic. Cannot update template_fields.") @staticmethod - def _configure_remote_target_path() -> Path | None: + def _configure_remote_target_path() -> tuple[Path, str] | tuple[None, None]: """Configure the remote target path if it is provided.""" if not remote_target_path: - return None + return None, None _configured_target_path = None @@ -270,7 +270,7 @@ def _configure_remote_target_path() -> Path | None: target_path_schema = target_path_str.split("://")[0] remote_conn_id = FILE_SCHEME_AIRFLOW_DEFAULT_CONN_ID_MAP.get(target_path_schema, None) # type: ignore[assignment] if remote_conn_id is None: - return _configured_target_path + return None, None if not AIRFLOW_IO_AVAILABLE: raise CosmosValueError( @@ -286,7 +286,7 @@ def _configure_remote_target_path() -> Path | None: if not _configured_target_path.exists(): # type: ignore[no-untyped-call] _configured_target_path.mkdir(parents=True, exist_ok=True) - return _configured_target_path + return _configured_target_path, remote_conn_id def upload_compiled_sql(self, tmp_project_dir: str, context: Context) -> None: """ @@ -295,7 +295,7 @@ def upload_compiled_sql(self, tmp_project_dir: str, context: Context) -> None: if not self.should_upload_compiled_sql: return - dest_target_dir = self._configure_remote_target_path() + dest_target_dir, dest_conn_id = self._configure_remote_target_path() if not dest_target_dir: raise CosmosValueError( "You're trying to upload compiled SQL files, but the remote target path is not configured. " @@ -303,9 +303,18 @@ def upload_compiled_sql(self, tmp_project_dir: str, context: Context) -> None: from airflow.io.path import ObjectStoragePath - source_target_dir = ObjectStoragePath(Path(tmp_project_dir) / "target" / "compiled") + source_compiled_dir = Path(tmp_project_dir) / "target" / "compiled" + files = [str(file) for file in source_compiled_dir.rglob("*") if file.is_file()] - source_target_dir.copy(dest_target_dir, recursive=True) # type: ignore[arg-type] + for file_path in files: + rel_path = os.path.relpath(file_path, source_compiled_dir) + + dest_path = ObjectStoragePath( + f"{str(dest_target_dir).rstrip('/')}/{context['dag'].dag_id}/{rel_path.lstrip('/')}", + conn_id=dest_conn_id, + ) + ObjectStoragePath(file_path).copy(dest_path) + self.log.debug("Copied %s to %s", file_path, dest_path) @provide_session def store_freshness_json(self, tmp_project_dir: str, context: Context, session: Session = NEW_SESSION) -> None: From 0ce662ea8832aaa63987fea37759106b7253a03a Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Mon, 30 Sep 2024 02:25:58 +0530 Subject: [PATCH 03/49] Add tests & minor refactorings --- .github/workflows/test.yml | 12 ++- cosmos/operators/airflow_async.py | 23 +---- cosmos/operators/local.py | 7 ++ dev/dags/simple_dag_async.py | 36 +++++++ tests/airflow/test_graph.py | 36 +++++++ tests/operators/test_airflow_async.py | 82 +++++++++++++++ tests/operators/test_base.py | 2 + tests/operators/test_local.py | 137 ++++++++++++++++++++++++++ tests/test_example_dags.py | 2 +- 9 files changed, 314 insertions(+), 23 deletions(-) create mode 100644 dev/dags/simple_dag_async.py create mode 100644 tests/operators/test_airflow_async.py diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index b0c33dfc6..a444dbc59 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -4,7 +4,7 @@ on: push: # Run on pushes to the default branch branches: [main] pull_request_target: # Also run on pull requests originated from forks - branches: [main] + branches: [main,poc-dbt-compile-task] concurrency: group: ${{ github.workflow }}-${{ github.head_ref || github.run_id }} @@ -176,6 +176,8 @@ jobs: POSTGRES_DB: postgres POSTGRES_SCHEMA: public POSTGRES_PORT: 5432 + AIRFLOW__COSMOS__REMOTE_TARGET_PATH: "s3://cosmos-remote-cache/target_compiled/" + AIRFLOW__COSMOS__REMOTE_TARGET_PATH_CONN_ID: aws_s3_conn - name: Upload coverage to Github uses: actions/upload-artifact@v4 @@ -248,6 +250,8 @@ jobs: POSTGRES_DB: postgres POSTGRES_SCHEMA: public POSTGRES_PORT: 5432 + AIRFLOW__COSMOS__REMOTE_TARGET_PATH: "s3://cosmos-remote-cache/target_compiled/" + AIRFLOW__COSMOS__REMOTE_TARGET_PATH_CONN_ID: aws_s3_conn - name: Upload coverage to Github uses: actions/upload-artifact@v4 @@ -316,6 +320,8 @@ jobs: POSTGRES_DB: postgres POSTGRES_SCHEMA: public POSTGRES_PORT: 5432 + AIRFLOW__COSMOS__REMOTE_TARGET_PATH: "s3://cosmos-remote-cache/target_compiled/" + AIRFLOW__COSMOS__REMOTE_TARGET_PATH_CONN_ID: aws_s3_conn - name: Upload coverage to Github uses: actions/upload-artifact@v4 @@ -393,6 +399,8 @@ jobs: POSTGRES_DB: postgres POSTGRES_SCHEMA: public POSTGRES_PORT: 5432 + AIRFLOW__COSMOS__REMOTE_TARGET_PATH: "s3://cosmos-remote-cache/target_compiled/" + AIRFLOW__COSMOS__REMOTE_TARGET_PATH_CONN_ID: aws_s3_conn - name: Upload coverage to Github uses: actions/upload-artifact@v4 @@ -537,6 +545,8 @@ jobs: POSTGRES_DB: postgres POSTGRES_SCHEMA: public POSTGRES_PORT: 5432 + AIRFLOW__COSMOS__REMOTE_TARGET_PATH: "s3://cosmos-remote-cache/target_compiled/" + AIRFLOW__COSMOS__REMOTE_TARGET_PATH_CONN_ID: aws_s3_conn - name: Upload coverage to Github uses: actions/upload-artifact@v4 diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index 334e074e5..05f762702 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -1,15 +1,10 @@ -from typing import Any - -from cosmos.operators.base import DbtCompileMixin from cosmos.operators.local import ( DbtBuildLocalOperator, - DbtDepsLocalOperator, + DbtCompileLocalOperator, DbtDocsAzureStorageLocalOperator, - DbtDocsCloudLocalOperator, DbtDocsGCSLocalOperator, DbtDocsLocalOperator, DbtDocsS3LocalOperator, - DbtLocalBaseOperator, DbtLSLocalOperator, DbtRunLocalOperator, DbtRunOperationLocalOperator, @@ -56,10 +51,6 @@ class DbtDocsAirflowAsyncOperator(DbtDocsLocalOperator): pass -class DbtDocsCloudAirflowAsyncOperator(DbtDocsCloudLocalOperator): - pass - - class DbtDocsS3AirflowAsyncOperator(DbtDocsS3LocalOperator): pass @@ -72,15 +63,5 @@ class DbtDocsGCSAirflowAsyncOperator(DbtDocsGCSLocalOperator): pass -class DbtDepsAirflowAsyncOperator(DbtDepsLocalOperator): +class DbtCompileAirflowAsyncOperator(DbtCompileLocalOperator): pass - - -class DbtCompileAirflowAsyncOperator(DbtCompileMixin, DbtLocalBaseOperator): - """ - Executes a dbt core build command. - """ - - def __init__(self, *args: Any, **kwargs: Any) -> None: - kwargs["should_upload_compiled_sql"] = True - super().__init__(*args, **kwargs) diff --git a/cosmos/operators/local.py b/cosmos/operators/local.py index 25b7d9dde..1083d5703 100644 --- a/cosmos/operators/local.py +++ b/cosmos/operators/local.py @@ -65,6 +65,7 @@ from cosmos.operators.base import ( AbstractDbtBaseOperator, DbtBuildMixin, + DbtCompileMixin, DbtLSMixin, DbtRunMixin, DbtRunOperationMixin, @@ -949,3 +950,9 @@ def __init__(self, **kwargs: str) -> None: raise DeprecationWarning( "The DbtDepsOperator has been deprecated. " "Please use the `install_deps` flag in dbt_args instead." ) + + +class DbtCompileLocalOperator(DbtCompileMixin, DbtLocalBaseOperator): + def __init__(self, *args: Any, **kwargs: Any) -> None: + kwargs["should_upload_compiled_sql"] = True + super().__init__(*args, **kwargs) diff --git a/dev/dags/simple_dag_async.py b/dev/dags/simple_dag_async.py new file mode 100644 index 000000000..d364ee6f2 --- /dev/null +++ b/dev/dags/simple_dag_async.py @@ -0,0 +1,36 @@ +import os +from datetime import datetime +from pathlib import Path + +from cosmos import DbtDag, ExecutionConfig, ExecutionMode, ProfileConfig, ProjectConfig +from cosmos.profiles import PostgresUserPasswordProfileMapping + +DEFAULT_DBT_ROOT_PATH = Path(__file__).parent / "dbt" +DBT_ROOT_PATH = Path(os.getenv("DBT_ROOT_PATH", DEFAULT_DBT_ROOT_PATH)) + +profile_config = ProfileConfig( + profile_name="default", + target_name="dev", + profile_mapping=PostgresUserPasswordProfileMapping( + conn_id="example_conn", + profile_args={"schema": "public"}, + disable_event_tracking=True, + ), +) + +simple_dag_async = DbtDag( + # dbt/cosmos-specific parameters + project_config=ProjectConfig( + DBT_ROOT_PATH / "jaffle_shop", + ), + profile_config=profile_config, + execution_config=ExecutionConfig( + execution_mode=ExecutionMode.AIRFLOW_ASYNC, + ), + # normal dag parameters + schedule_interval=None, + start_date=datetime(2023, 1, 1), + catchup=False, + dag_id="simple_dag_async", + tags=["simple"], +) diff --git a/tests/airflow/test_graph.py b/tests/airflow/test_graph.py index 72a09a5e5..d864b73e4 100644 --- a/tests/airflow/test_graph.py +++ b/tests/airflow/test_graph.py @@ -30,6 +30,7 @@ from cosmos.converter import airflow_kwargs from cosmos.dbt.graph import DbtNode from cosmos.profiles import PostgresUserPasswordProfileMapping +from cosmos.settings import dbt_compile_task_id SAMPLE_PROJ_PATH = Path("/home/user/path/dbt-proj/") SOURCE_RENDERING_BEHAVIOR = SourceRenderingBehavior(os.getenv("SOURCE_RENDERING_BEHAVIOR", "none")) @@ -226,6 +227,41 @@ def test_build_airflow_graph_with_after_all(): assert dag.leaves[0].select == ["tag:some"] +@pytest.mark.integration +def test_build_airflow_graph_with_dbt_compile_task(): + with DAG("test-id-dbt-compile", start_date=datetime(2022, 1, 1)) as dag: + task_args = { + "project_dir": SAMPLE_PROJ_PATH, + "conn_id": "fake_conn", + "profile_config": ProfileConfig( + profile_name="default", + target_name="default", + profile_mapping=PostgresUserPasswordProfileMapping( + conn_id="fake_conn", + profile_args={"schema": "public"}, + ), + ), + } + render_config = RenderConfig( + select=["tag:some"], + test_behavior=TestBehavior.AFTER_ALL, + source_rendering_behavior=SOURCE_RENDERING_BEHAVIOR, + ) + build_airflow_graph( + nodes=sample_nodes, + dag=dag, + execution_mode=ExecutionMode.AIRFLOW_ASYNC, + test_indirect_selection=TestIndirectSelection.EAGER, + task_args=task_args, + dbt_project_name="astro_shop", + render_config=render_config, + ) + + task_ids = [task.task_id for task in dag.tasks] + assert dbt_compile_task_id in task_ids + assert dbt_compile_task_id in dag.tasks[0].upstream_task_ids + + def test_calculate_operator_class(): class_module_import_path = calculate_operator_class(execution_mode=ExecutionMode.KUBERNETES, dbt_class="DbtSeed") assert class_module_import_path == "cosmos.operators.kubernetes.DbtSeedKubernetesOperator" diff --git a/tests/operators/test_airflow_async.py b/tests/operators/test_airflow_async.py new file mode 100644 index 000000000..fc085c7d0 --- /dev/null +++ b/tests/operators/test_airflow_async.py @@ -0,0 +1,82 @@ +from cosmos.operators.airflow_async import ( + DbtBuildAirflowAsyncOperator, + DbtCompileAirflowAsyncOperator, + DbtDocsAirflowAsyncOperator, + DbtDocsAzureStorageAirflowAsyncOperator, + DbtDocsGCSAirflowAsyncOperator, + DbtDocsS3AirflowAsyncOperator, + DbtLSAirflowAsyncOperator, + DbtRunAirflowAsyncOperator, + DbtRunOperationAirflowAsyncOperator, + DbtSeedAirflowAsyncOperator, + DbtSnapshotAirflowAsyncOperator, + DbtSourceAirflowAsyncOperator, + DbtTestAirflowAsyncOperator, +) +from cosmos.operators.local import ( + DbtBuildLocalOperator, + DbtCompileLocalOperator, + DbtDocsAzureStorageLocalOperator, + DbtDocsGCSLocalOperator, + DbtDocsLocalOperator, + DbtDocsS3LocalOperator, + DbtLSLocalOperator, + DbtRunLocalOperator, + DbtRunOperationLocalOperator, + DbtSeedLocalOperator, + DbtSnapshotLocalOperator, + DbtSourceLocalOperator, + DbtTestLocalOperator, +) + + +def test_dbt_build_airflow_async_operator_inheritance(): + assert issubclass(DbtBuildAirflowAsyncOperator, DbtBuildLocalOperator) + + +def test_dbt_ls_airflow_async_operator_inheritance(): + assert issubclass(DbtLSAirflowAsyncOperator, DbtLSLocalOperator) + + +def test_dbt_seed_airflow_async_operator_inheritance(): + assert issubclass(DbtSeedAirflowAsyncOperator, DbtSeedLocalOperator) + + +def test_dbt_snapshot_airflow_async_operator_inheritance(): + assert issubclass(DbtSnapshotAirflowAsyncOperator, DbtSnapshotLocalOperator) + + +def test_dbt_source_airflow_async_operator_inheritance(): + assert issubclass(DbtSourceAirflowAsyncOperator, DbtSourceLocalOperator) + + +def test_dbt_run_airflow_async_operator_inheritance(): + assert issubclass(DbtRunAirflowAsyncOperator, DbtRunLocalOperator) + + +def test_dbt_test_airflow_async_operator_inheritance(): + assert issubclass(DbtTestAirflowAsyncOperator, DbtTestLocalOperator) + + +def test_dbt_run_operation_airflow_async_operator_inheritance(): + assert issubclass(DbtRunOperationAirflowAsyncOperator, DbtRunOperationLocalOperator) + + +def test_dbt_docs_airflow_async_operator_inheritance(): + assert issubclass(DbtDocsAirflowAsyncOperator, DbtDocsLocalOperator) + + +def test_dbt_docs_s3_airflow_async_operator_inheritance(): + assert issubclass(DbtDocsS3AirflowAsyncOperator, DbtDocsS3LocalOperator) + + +def test_dbt_docs_azure_storage_airflow_async_operator_inheritance(): + assert issubclass(DbtDocsAzureStorageAirflowAsyncOperator, DbtDocsAzureStorageLocalOperator) + + +def test_dbt_docs_gcs_airflow_async_operator_inheritance(): + assert issubclass(DbtDocsGCSAirflowAsyncOperator, DbtDocsGCSLocalOperator) + + +def test_dbt_compile_airflow_async_operator_inheritance(): + assert issubclass(DbtCompileAirflowAsyncOperator, DbtCompileLocalOperator) diff --git a/tests/operators/test_base.py b/tests/operators/test_base.py index 6f4425282..e97c2d396 100644 --- a/tests/operators/test_base.py +++ b/tests/operators/test_base.py @@ -8,6 +8,7 @@ from cosmos.operators.base import ( AbstractDbtBaseOperator, DbtBuildMixin, + DbtCompileMixin, DbtLSMixin, DbtRunMixin, DbtRunOperationMixin, @@ -143,6 +144,7 @@ def test_dbt_base_operator_context_merge( ("seed", DbtSeedMixin), ("run", DbtRunMixin), ("build", DbtBuildMixin), + ("compile", DbtCompileMixin), ], ) def test_dbt_mixin_base_cmd(dbt_command, dbt_operator_class): diff --git a/tests/operators/test_local.py b/tests/operators/test_local.py index d54bbb5e1..fa3d87e4f 100644 --- a/tests/operators/test_local.py +++ b/tests/operators/test_local.py @@ -25,9 +25,11 @@ parse_number_of_warnings_dbt_runner, parse_number_of_warnings_subprocess, ) +from cosmos.exceptions import CosmosValueError from cosmos.hooks.subprocess import FullOutputSubprocessResult from cosmos.operators.local import ( DbtBuildLocalOperator, + DbtCompileLocalOperator, DbtDocsAzureStorageLocalOperator, DbtDocsGCSLocalOperator, DbtDocsLocalOperator, @@ -42,6 +44,7 @@ DbtTestLocalOperator, ) from cosmos.profiles import PostgresUserPasswordProfileMapping +from cosmos.settings import AIRFLOW_IO_AVAILABLE from tests.utils import test_dag as run_test_dag DBT_PROJ_DIR = Path(__file__).parent.parent.parent / "dev/dags/dbt/jaffle_shop" @@ -1052,3 +1055,137 @@ def test_store_freshness_not_store_compiled_sql(mock_context, mock_session): # Verify the freshness attribute is set correctly assert instance.freshness == "" + + +def test_dbt_compile_local_operator_initialisation(): + operator = DbtCompileLocalOperator( + task_id="fake-task", + profile_config=profile_config, + project_dir="fake-dir", + ) + assert operator.should_upload_compiled_sql is True + assert "compile" in operator.base_cmd + + +@patch("cosmos.operators.local.remote_target_path", new="s3://some-bucket/target") +@patch("cosmos.operators.local.AIRFLOW_IO_AVAILABLE", new=False) +def test_configure_remote_target_path_object_storage_unavailable_on_earlier_airflow_versions(): + operator = DbtCompileLocalOperator( + task_id="fake-task", + profile_config=profile_config, + project_dir="fake-dir", + ) + with pytest.raises(CosmosValueError, match="Object Storage feature is unavailable"): + operator._configure_remote_target_path() + + +@pytest.mark.parametrize( + "rem_target_path, rem_target_path_conn_id", + [ + (None, "aws_s3_conn"), + ("unknown://some-bucket/cache", None), + ], +) +def test_config_remote_target_path_unset_settings(rem_target_path, rem_target_path_conn_id): + with patch("cosmos.operators.local.remote_target_path", new=rem_target_path): + with patch("cosmos.operators.local.remote_target_path_conn_id", new=rem_target_path_conn_id): + operator = DbtCompileLocalOperator( + task_id="fake-task", + profile_config=profile_config, + project_dir="fake-dir", + ) + target_path, target_conn = operator._configure_remote_target_path() + assert target_path is None + assert target_conn is None + + +@pytest.mark.skipif(not AIRFLOW_IO_AVAILABLE, reason="Airflow did not have Object Storage until the 2.8 release") +@patch("cosmos.operators.local.remote_target_path", new="s3://some-bucket/target") +@patch("cosmos.operators.local.remote_target_path_conn_id", new="aws_s3_conn") +@patch("airflow.io.path.ObjectStoragePath") +def test_configure_remote_target_path(mock_object_storage_path): + operator = DbtCompileLocalOperator( + task_id="fake-task", + profile_config=profile_config, + project_dir="fake-dir", + ) + mock_remote_path = MagicMock() + mock_object_storage_path.return_value.exists.return_value = True + mock_object_storage_path.return_value = mock_remote_path + target_path, target_conn = operator._configure_remote_target_path() + assert target_path == mock_remote_path + assert target_conn == "aws_s3_conn" + mock_object_storage_path.assert_called_with("s3://some-bucket/target", conn_id="aws_s3_conn") + + mock_object_storage_path.return_value.exists.return_value = False + mock_object_storage_path.return_value.mkdir.return_value = MagicMock() + _, _ = operator._configure_remote_target_path() + mock_object_storage_path.return_value.mkdir.assert_called_with(parents=True, exist_ok=True) + + +@patch.object(DbtLocalBaseOperator, "_configure_remote_target_path") +def test_no_compiled_sql_upload_for_other_operators(mock_configure_remote_target_path): + operator = DbtSeedLocalOperator( + task_id="fake-task", + profile_config=profile_config, + project_dir="fake-dir", + ) + assert operator.should_upload_compiled_sql is False + operator.upload_compiled_sql("fake-dir", MagicMock()) + mock_configure_remote_target_path.assert_not_called() + + +@patch("cosmos.operators.local.DbtCompileLocalOperator._configure_remote_target_path") +def test_upload_compiled_sql_no_remote_path_raises_error(mock_configure_remote): + operator = DbtCompileLocalOperator( + task_id="fake-task", + profile_config=profile_config, + project_dir="fake-dir", + ) + + mock_configure_remote.return_value = (None, None) + + tmp_project_dir = "/fake/tmp/project" + context = {"dag": MagicMock(dag_id="test_dag")} + + with pytest.raises(CosmosValueError, match="remote target path is not configured"): + operator.upload_compiled_sql(tmp_project_dir, context) + + +@pytest.mark.skipif(not AIRFLOW_IO_AVAILABLE, reason="Airflow did not have Object Storage until the 2.8 release") +@patch("airflow.io.path.ObjectStoragePath.copy") +@patch("airflow.io.path.ObjectStoragePath") +@patch("cosmos.operators.local.DbtCompileLocalOperator._configure_remote_target_path") +def test_upload_compiled_sql_should_upload(mock_configure_remote, mock_object_storage_path, mock_copy): + """Test upload_compiled_sql when should_upload_compiled_sql is True and uploads files.""" + operator = DbtCompileLocalOperator( + task_id="fake-task", + profile_config=profile_config, + project_dir="fake-dir", + ) + + mock_configure_remote.return_value = ("mock_remote_path", "mock_conn_id") + + tmp_project_dir = "/fake/tmp/project" + source_compiled_dir = Path(tmp_project_dir) / "target" / "compiled" + + file1 = MagicMock(spec=Path) + file1.is_file.return_value = True + file1.__str__.return_value = str(source_compiled_dir / "file1.sql") + + file2 = MagicMock(spec=Path) + file2.is_file.return_value = True + file2.__str__.return_value = str(source_compiled_dir / "file2.sql") + + files = [file1, file2] + + with patch.object(Path, "rglob", return_value=files): + context = {"dag": MagicMock(dag_id="test_dag")} + + operator.upload_compiled_sql(tmp_project_dir, context) + + for file_path in files: + rel_path = os.path.relpath(str(file_path), str(source_compiled_dir)) + expected_dest_path = f"mock_remote_path/test_dag/{rel_path.lstrip('/')}" + mock_object_storage_path.assert_any_call(expected_dest_path, conn_id="mock_conn_id") + mock_object_storage_path.return_value.copy.assert_any_call(mock_object_storage_path.return_value) diff --git a/tests/test_example_dags.py b/tests/test_example_dags.py index 9f8601156..9aa66432d 100644 --- a/tests/test_example_dags.py +++ b/tests/test_example_dags.py @@ -28,7 +28,7 @@ MIN_VER_DAG_FILE: dict[str, list[str]] = { "2.4": ["cosmos_seed_dag.py"], - "2.8": ["cosmos_manifest_example.py"], + "2.8": ["cosmos_manifest_example.py", "simple_dag_async.py"], } IGNORED_DAG_FILES = ["performance_dag.py", "jaffle_shop_kubernetes.py"] From 1b6f57e1806c0e1d9d3a4d830d752906aefaa1ef Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Mon, 30 Sep 2024 02:40:58 +0530 Subject: [PATCH 04/49] Apply suggestions from code review --- .github/workflows/test.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index a444dbc59..baa535d4b 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -2,9 +2,9 @@ name: test on: push: # Run on pushes to the default branch - branches: [main] - pull_request_target: # Also run on pull requests originated from forks branches: [main,poc-dbt-compile-task] + pull_request_target: # Also run on pull requests originated from forks + branches: [main] concurrency: group: ${{ github.workflow }}-${{ github.head_ref || github.run_id }} From cc48161f9ca4af0efda37c958c4486549424bbe0 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Mon, 30 Sep 2024 03:37:35 +0530 Subject: [PATCH 05/49] Install deps for the newly added example DAG --- dev/dags/simple_dag_async.py | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/dags/simple_dag_async.py b/dev/dags/simple_dag_async.py index d364ee6f2..cf5f57e02 100644 --- a/dev/dags/simple_dag_async.py +++ b/dev/dags/simple_dag_async.py @@ -33,4 +33,5 @@ catchup=False, dag_id="simple_dag_async", tags=["simple"], + operator_args={"install_deps": True}, ) From 106802547d4bb69d807e7cdba806fa33f3127b36 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Mon, 30 Sep 2024 12:02:13 +0530 Subject: [PATCH 06/49] Add docs --- dev/dags/simple_dag_async.py | 2 ++ docs/configuration/cosmos-conf.rst | 21 ++++++++++++ docs/getting_started/execution-modes.rst | 41 +++++++++++++++++++++++- 3 files changed, 63 insertions(+), 1 deletion(-) diff --git a/dev/dags/simple_dag_async.py b/dev/dags/simple_dag_async.py index cf5f57e02..787461236 100644 --- a/dev/dags/simple_dag_async.py +++ b/dev/dags/simple_dag_async.py @@ -18,6 +18,7 @@ ), ) +# [START airflow_async_execution_mode_example] simple_dag_async = DbtDag( # dbt/cosmos-specific parameters project_config=ProjectConfig( @@ -35,3 +36,4 @@ tags=["simple"], operator_args={"install_deps": True}, ) +# [END airflow_async_execution_mode_example] diff --git a/docs/configuration/cosmos-conf.rst b/docs/configuration/cosmos-conf.rst index 95a4adcad..8b4e9fed9 100644 --- a/docs/configuration/cosmos-conf.rst +++ b/docs/configuration/cosmos-conf.rst @@ -126,6 +126,27 @@ This page lists all available Airflow configurations that affect ``astronomer-co - Default: ``None`` - Environment Variable: ``AIRFLOW__COSMOS__REMOTE_CACHE_DIR_CONN_ID`` +.. _remote_target_path: + +`remote_target_path`_: + (Introduced since Cosmos 1.7.0) The path to the remote target directory. This is the directory designated to + remotely copy & store in the files generated and stored by dbt in the dbt project's target directory. The value + for the remote target path can be any of the schemes that are supported by the + `Airflow Object Store `_ + feature introduced in Airflow 2.8.0 (e.g. ``s3://your_s3_bucket/cache_dir/``, ``gs://your_gs_bucket/cache_dir/``, + ``abfs://your_azure_container/cache_dir``, etc.) + + - Default: ``None`` + - Environment Variable: ``AIRFLOW__COSMOS__REMOTE_TARGET_PATH`` + +.. _remote_target_path_conn_id: + +`remote_target_path_conn_id`_: + (Introduced since Cosmos 1.7.0) The connection ID for the remote target path. If this is not set, the default + Airflow connection ID identified for the scheme will be used. + + - Default: ``None`` + - Environment Variable: ``AIRFLOW__COSMOS__REMOTE_TARGET_PATH_CONN_ID`` [openlineage] ~~~~~~~~~~~~~ diff --git a/docs/getting_started/execution-modes.rst b/docs/getting_started/execution-modes.rst index f80c3da9d..ecd60c0b0 100644 --- a/docs/getting_started/execution-modes.rst +++ b/docs/getting_started/execution-modes.rst @@ -12,12 +12,13 @@ Cosmos can run ``dbt`` commands using five different approaches, called ``execut 5. **aws_eks**: Run ``dbt`` commands from AWS EKS Pods managed by Cosmos (requires a pre-existing Docker image) 6. **azure_container_instance**: Run ``dbt`` commands from Azure Container Instances managed by Cosmos (requires a pre-existing Docker image) 7. **gcp_cloud_run_job**: Run ``dbt`` commands from GCP Cloud Run Job instances managed by Cosmos (requires a pre-existing Docker image) +8. **airflow_async**: (Introduced since Cosmos 1.7.0) Run the dbt resources from your dbt project asynchronously, by submitting the corresponding compiled SQLs to Apache Airflow's `Deferrable operators `__ The choice of the ``execution mode`` can vary based on each user's needs and concerns. For more details, check each execution mode described below. .. list-table:: Execution Modes Comparison - :widths: 20 20 20 20 20 + :widths: 25 25 25 25 :header-rows: 1 * - Execution Mode @@ -52,6 +53,10 @@ The choice of the ``execution mode`` can vary based on each user's needs and con - Slow - High - No + * - Airflow Async + - Medium + - None + - Yes Local ----- @@ -238,6 +243,40 @@ Each task will create a new Cloud Run Job execution, giving full isolation. The }, ) +Airflow Async +------------- + +.. versionadded:: 1.7.0 + +(**Experimental**) + +The ``airflow_async`` execution mode is a way to run the dbt resources from your dbt project using Apache Airflow's +`Deferrable operators `__. +This execution mode could be preferred when you've long running resources and you want to run them asynchronously by +leveraging Airflow's deferrable operators. With that, you would be able to potentially observe higher throughput of tasks +as more dbt nodes will be run in parallel since they won't be blocking Airflow's worker slots. + +In this mode, Cosmos adds a new operator, ``DbtCompileAirflowAsyncOperator``, as a root task in the DAG. The task runs +the ``dbt compile`` command on your dbt project which then outputs compiled SQLs in the project's target directory. +As part of the same task run, these compiled SQLs are then stored remotely to a remote path set using the +:ref:`remote_target_path` configuration. The remote path is then used by the subsequent tasks in the DAG to +fetch (from the remote path) and run the compiled SQLs asynchronously using e.g. the ``DbtRunAirflowAsyncOperator``. +You may observe that the compile task takes a bit longer to run due to the latency of storing the compiled SQLs remotely, +however, it is still a win as it is one-time overhead and the subsequent tasks run asynchronously utilising the Airflow's +deferrable operators and supplying to them those compiled SQLs. + +Note that currently, the ``airflow_async`` execution mode has the following limitations and is released as Experimental: + +1. Only supports the ``dbt resource type`` models to be run asynchronously using Airflow deferrable operators. All other resources are executed synchronously using dbt commands as they are in the ``local`` execution mode. +2. Only supports BigQuery as the target database. If a profile target other than BigQuery is specified, Cosmos will error out saying that the target database is not supported with this execution mode. + +Example DAG: + +.. literalinclude:: ../../dev/dags/simple_dag_async.py + :language: python + :start-after: [START airflow_async_execution_mode_example] + :end-before: [END airflow_async_execution_mode_example] + .. _invocation_modes: Invocation Modes ================ From faa706d55c71c0a7da7daddb92dc393f36fb5e19 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 25 Sep 2024 13:26:24 +0530 Subject: [PATCH 07/49] Add async run operator Remove print stmt Fix query Fix query Remove oss execute method code --- cosmos/config.py | 14 ++++++ cosmos/converter.py | 3 ++ cosmos/core/airflow.py | 7 ++- cosmos/core/graph/entities.py | 1 + cosmos/dbt/graph.py | 3 +- cosmos/operators/airflow_async.py | 76 +++++++++++++++++++++++++++++-- cosmos/operators/base.py | 2 + cosmos/operators/local.py | 4 ++ 8 files changed, 103 insertions(+), 7 deletions(-) diff --git a/cosmos/config.py b/cosmos/config.py index 2cebbf3cc..b92a25a20 100644 --- a/cosmos/config.py +++ b/cosmos/config.py @@ -10,6 +10,7 @@ from pathlib import Path from typing import Any, Callable, Iterator +import yaml from airflow.version import version as airflow_version from cosmos.cache import create_cache_profile, get_cached_profile, is_profile_cache_enabled @@ -286,6 +287,19 @@ def validate_profiles_yml(self) -> None: if self.profiles_yml_filepath and not Path(self.profiles_yml_filepath).exists(): raise CosmosValueError(f"The file {self.profiles_yml_filepath} does not exist.") + def get_profile_type(self): + if self.profile_mapping.dbt_profile_type: + return self.profile_mapping.dbt_profile_type + + profile_path = self._get_profile_path() + + with open(profile_path) as file: + profiles = yaml.safe_load(file) + + profile = profiles[self.profile_name] + target_type = profile["outputs"][self.target_name]["type"] + return target_type + def _get_profile_path(self, use_mock_values: bool = False) -> Path: """ Handle the profile caching mechanism. diff --git a/cosmos/converter.py b/cosmos/converter.py index fd077c465..e4e0b7f6b 100644 --- a/cosmos/converter.py +++ b/cosmos/converter.py @@ -207,6 +207,7 @@ def __init__( task_group: TaskGroup | None = None, operator_args: dict[str, Any] | None = None, on_warning_callback: Callable[..., Any] | None = None, + async_op_args: dict[str, Any] | None = None, *args: Any, **kwargs: Any, ) -> None: @@ -256,6 +257,7 @@ def __init__( cache_identifier=cache_identifier, dbt_vars=dbt_vars, airflow_metadata=cache._get_airflow_metadata(dag, task_group), + async_op_args=async_op_args, ) self.dbt_graph.load(method=render_config.load_method, execution_mode=execution_config.execution_mode) @@ -301,6 +303,7 @@ def __init__( dbt_project_name=project_config.project_name, on_warning_callback=on_warning_callback, render_config=render_config, + async_op_args=async_op_args, ) current_time = time.perf_counter() diff --git a/cosmos/core/airflow.py b/cosmos/core/airflow.py index 9e1d08ac1..67230fb39 100644 --- a/cosmos/core/airflow.py +++ b/cosmos/core/airflow.py @@ -1,3 +1,5 @@ +from __future__ import annotations + import importlib from airflow.models import BaseOperator @@ -10,7 +12,7 @@ logger = get_logger(__name__) -def get_airflow_task(task: Task, dag: DAG, task_group: "TaskGroup | None" = None) -> BaseOperator: +def get_airflow_task(task: Task, dag: DAG, task_group: TaskGroup | None = None) -> BaseOperator: """ Get the Airflow Operator class for a Task. @@ -29,6 +31,9 @@ def get_airflow_task(task: Task, dag: DAG, task_group: "TaskGroup | None" = None if task.owner != "": task_kwargs["owner"] = task.owner + if task.async_op_args: + task_kwargs["async_op_args"] = task.async_op_args + airflow_task = Operator( task_id=task.id, dag=dag, diff --git a/cosmos/core/graph/entities.py b/cosmos/core/graph/entities.py index 6bf9ff046..340cb7781 100644 --- a/cosmos/core/graph/entities.py +++ b/cosmos/core/graph/entities.py @@ -61,3 +61,4 @@ class Task(CosmosEntity): operator_class: str = "airflow.operators.empty.EmptyOperator" arguments: Dict[str, Any] = field(default_factory=dict) extra_context: Dict[str, Any] = field(default_factory=dict) + async_op_args: Dict[str, Any] = field(default_factory=dict) diff --git a/cosmos/dbt/graph.py b/cosmos/dbt/graph.py index 1c0237e8f..0400edc1a 100644 --- a/cosmos/dbt/graph.py +++ b/cosmos/dbt/graph.py @@ -217,6 +217,7 @@ def __init__( dbt_vars: dict[str, str] | None = None, airflow_metadata: dict[str, str] | None = None, operator_args: dict[str, Any] | None = None, + async_op_args: dict[str, Any] | None = None, ): self.project = project self.render_config = render_config @@ -224,6 +225,7 @@ def __init__( self.execution_config = execution_config self.cache_dir = cache_dir self.airflow_metadata = airflow_metadata or {} + self.async_op_args = async_op_args if cache_identifier: self.dbt_ls_cache_key = cache.create_cache_key(cache_identifier) else: @@ -467,7 +469,6 @@ def should_use_dbt_ls_cache(self) -> bool: def load_via_dbt_ls_cache(self) -> bool: """(Try to) load dbt ls cache from an Airflow Variable""" - logger.info(f"Trying to parse the dbt project using dbt ls cache {self.dbt_ls_cache_key}...") if self.should_use_dbt_ls_cache(): project_path = self.project_path diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index 05f762702..f278dcba8 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -1,19 +1,34 @@ +from __future__ import annotations + +from typing import Any + +from airflow.io.path import ObjectStoragePath +from airflow.providers.google.cloud.operators.bigquery import BigQueryInsertJobOperator +from airflow.utils.context import Context + +from cosmos.operators.base import DbtCompileMixin from cosmos.operators.local import ( DbtBuildLocalOperator, - DbtCompileLocalOperator, + DbtDepsLocalOperator, DbtDocsAzureStorageLocalOperator, + DbtDocsCloudLocalOperator, DbtDocsGCSLocalOperator, DbtDocsLocalOperator, DbtDocsS3LocalOperator, + DbtLocalBaseOperator, DbtLSLocalOperator, - DbtRunLocalOperator, - DbtRunOperationLocalOperator, DbtSeedLocalOperator, DbtSnapshotLocalOperator, DbtSourceLocalOperator, DbtTestLocalOperator, + DbtRunOperationLocalOperator, + DbtCompileLocalOperator, ) +from cosmos.settings import remote_target_path, remote_target_path_conn_id + +_SUPPORTED_DATABASES = ["bigquery"] + class DbtBuildAirflowAsyncOperator(DbtBuildLocalOperator): pass @@ -35,8 +50,50 @@ class DbtSourceAirflowAsyncOperator(DbtSourceLocalOperator): pass -class DbtRunAirflowAsyncOperator(DbtRunLocalOperator): - pass +class DbtRunAirflowAsyncOperator(BigQueryInsertJobOperator): + def __init__(self, *args, **kwargs): + self.configuration = {} + self.job_id = kwargs.get("job_id", {}) or "" + self.impersonation_chain = kwargs.get("impersonation_chain", {}) or "" + self.project_id = kwargs.get("project_id", {}) or "" + + self.profile_config = kwargs.get("profile_config") + self.project_dir = kwargs.get("project_dir") + + self.async_op_args = kwargs.get("async_op_args", {}) + self.async_op_args["deferrable"] = True + super().__init__(*args, configuration=self.configuration, task_id=kwargs.get("task_id"), **self.async_op_args) + self.profile_type = self.profile_config.get_profile_type() + if self.profile_type not in _SUPPORTED_DATABASES: + raise f"Async run are only supported: {_SUPPORTED_DATABASES}" + + self.reattach_states: set[str] = self.async_op_args.get("reattach_states") or set() + + def get_remote_sql(self): + project_name = str(self.project_dir).split("/")[-1] + model_name: str = self.task_id.split(".")[0] + if model_name.startswith("stg_"): + remote_model_path = f"{remote_target_path}/{project_name}/models/staging/{model_name}.sql" + else: + remote_model_path = f"{remote_target_path}/{project_name}/models/{model_name}.sql" + + print("remote_model_path: ", remote_model_path) + object_storage_path = ObjectStoragePath(remote_model_path, conn_id=remote_target_path_conn_id) + with object_storage_path.open() as fp: + return fp.read() + + def execute(self, context: Context) -> Any | None: + sql = self.get_remote_sql() + print("sql: ", sql) + self.configuration = { + "query": { + "query": sql, + "useLegacySql": False, + } + } + print("async_op_args: ", self.async_op_args) + super().execute(context) + class DbtTestAirflowAsyncOperator(DbtTestLocalOperator): @@ -51,6 +108,10 @@ class DbtDocsAirflowAsyncOperator(DbtDocsLocalOperator): pass +class DbtDocsCloudAirflowAsyncOperator(DbtDocsCloudLocalOperator): + pass + + class DbtDocsS3AirflowAsyncOperator(DbtDocsS3LocalOperator): pass @@ -65,3 +126,8 @@ class DbtDocsGCSAirflowAsyncOperator(DbtDocsGCSLocalOperator): class DbtCompileAirflowAsyncOperator(DbtCompileLocalOperator): pass + +class DbtDepsAirflowAsyncOperator(DbtDepsLocalOperator): + pass + + diff --git a/cosmos/operators/base.py b/cosmos/operators/base.py index ed7969ebd..45f8bd173 100644 --- a/cosmos/operators/base.py +++ b/cosmos/operators/base.py @@ -110,6 +110,7 @@ def __init__( dbt_cmd_global_flags: list[str] | None = None, cache_dir: Path | None = None, extra_context: dict[str, Any] | None = None, + # configuration: dict[str, Any] | None = None, **kwargs: Any, ) -> None: self.project_dir = project_dir @@ -140,6 +141,7 @@ def __init__( self.cache_dir = cache_dir self.extra_context = extra_context or {} kwargs.pop("full_refresh", None) # usage of this param should be implemented in child classes + # kwargs["configuration"] = {} super().__init__(**kwargs) def get_env(self, context: Context) -> dict[str, str | bytes | os.PathLike[Any]]: diff --git a/cosmos/operators/local.py b/cosmos/operators/local.py index 1083d5703..ad320d5fc 100644 --- a/cosmos/operators/local.py +++ b/cosmos/operators/local.py @@ -135,6 +135,7 @@ def __init__( should_store_compiled_sql: bool = True, should_upload_compiled_sql: bool = False, append_env: bool = True, + async_op_args: dict[str, Any] | None = None, **kwargs: Any, ) -> None: self.profile_config = profile_config @@ -148,6 +149,7 @@ def __init__( self.invoke_dbt: Callable[..., FullOutputSubprocessResult | dbtRunnerResult] self.handle_exception: Callable[..., None] self._dbt_runner: dbtRunner | None = None + self.async_op_args = async_op_args if self.invocation_mode: self._set_invocation_methods() super().__init__(**kwargs) @@ -289,6 +291,7 @@ def _configure_remote_target_path() -> tuple[Path, str] | tuple[None, None]: return _configured_target_path, remote_conn_id + def upload_compiled_sql(self, tmp_project_dir: str, context: Context) -> None: """ Uploads the compiled SQL files from the dbt compile output to the remote store. @@ -297,6 +300,7 @@ def upload_compiled_sql(self, tmp_project_dir: str, context: Context) -> None: return dest_target_dir, dest_conn_id = self._configure_remote_target_path() + if not dest_target_dir: raise CosmosValueError( "You're trying to upload compiled SQL files, but the remote target path is not configured. " From 0e155e41a6d947f0d9988f2363bfd0e81f859ae7 Mon Sep 17 00:00:00 2001 From: pankajastro Date: Mon, 30 Sep 2024 15:51:44 +0530 Subject: [PATCH 08/49] Fix remote sql path and async args --- cosmos/airflow/graph.py | 4 ++++ cosmos/operators/airflow_async.py | 15 +++++---------- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/cosmos/airflow/graph.py b/cosmos/airflow/graph.py index d84a1fafb..8edc9b232 100644 --- a/cosmos/airflow/graph.py +++ b/cosmos/airflow/graph.py @@ -81,6 +81,7 @@ def create_test_task_metadata( on_warning_callback: Callable[..., Any] | None = None, node: DbtNode | None = None, render_config: RenderConfig | None = None, + async_op_args: dict | None = None, ) -> TaskMetadata: """ Create the metadata that will be used to instantiate the Airflow Task that will be used to run the Dbt test node. @@ -125,6 +126,7 @@ def create_test_task_metadata( ), arguments=task_args, extra_context=extra_context, + async_op_args=async_op_args, ) @@ -287,6 +289,7 @@ def build_airflow_graph( render_config: RenderConfig, task_group: TaskGroup | None = None, on_warning_callback: Callable[..., Any] | None = None, # argument specific to the DBT test command + async_op_args: dict | None = None, ) -> None: """ Instantiate dbt `nodes` as Airflow tasks within the given `task_group` (optional) or `dag` (mandatory). @@ -351,6 +354,7 @@ def build_airflow_graph( task_args=task_args, on_warning_callback=on_warning_callback, render_config=render_config, + async_op_args=async_op_args, ) test_task = create_airflow_task(test_meta, dag, task_group=task_group) leaves_ids = calculate_leaves(tasks_ids=list(tasks_map.keys()), nodes=nodes) diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index f278dcba8..4f1fae227 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -6,25 +6,22 @@ from airflow.providers.google.cloud.operators.bigquery import BigQueryInsertJobOperator from airflow.utils.context import Context -from cosmos.operators.base import DbtCompileMixin from cosmos.operators.local import ( DbtBuildLocalOperator, + DbtCompileLocalOperator, DbtDepsLocalOperator, DbtDocsAzureStorageLocalOperator, DbtDocsCloudLocalOperator, DbtDocsGCSLocalOperator, DbtDocsLocalOperator, DbtDocsS3LocalOperator, - DbtLocalBaseOperator, DbtLSLocalOperator, + DbtRunOperationLocalOperator, DbtSeedLocalOperator, DbtSnapshotLocalOperator, DbtSourceLocalOperator, DbtTestLocalOperator, - DbtRunOperationLocalOperator, - DbtCompileLocalOperator, ) - from cosmos.settings import remote_target_path, remote_target_path_conn_id _SUPPORTED_DATABASES = ["bigquery"] @@ -73,9 +70,9 @@ def get_remote_sql(self): project_name = str(self.project_dir).split("/")[-1] model_name: str = self.task_id.split(".")[0] if model_name.startswith("stg_"): - remote_model_path = f"{remote_target_path}/{project_name}/models/staging/{model_name}.sql" + remote_model_path = f"{remote_target_path}/{self.dag_id}/{project_name}/models/staging/{model_name}.sql" else: - remote_model_path = f"{remote_target_path}/{project_name}/models/{model_name}.sql" + remote_model_path = f"{remote_target_path}/{self.dag_id}/{project_name}/models/{model_name}.sql" print("remote_model_path: ", remote_model_path) object_storage_path = ObjectStoragePath(remote_model_path, conn_id=remote_target_path_conn_id) @@ -95,7 +92,6 @@ def execute(self, context: Context) -> Any | None: super().execute(context) - class DbtTestAirflowAsyncOperator(DbtTestLocalOperator): pass @@ -127,7 +123,6 @@ class DbtDocsGCSAirflowAsyncOperator(DbtDocsGCSLocalOperator): class DbtCompileAirflowAsyncOperator(DbtCompileLocalOperator): pass + class DbtDepsAirflowAsyncOperator(DbtDepsLocalOperator): pass - - From 5f1ecaac695b2fafff28771fb67e9bf62833dadb Mon Sep 17 00:00:00 2001 From: pankajastro Date: Mon, 30 Sep 2024 19:58:16 +0530 Subject: [PATCH 09/49] Fix query --- cosmos/operators/airflow_async.py | 62 ++++++++++++++----- .../jaffle_shop/models/staging/sources.yml | 31 ---------- 2 files changed, 46 insertions(+), 47 deletions(-) delete mode 100644 dev/dags/dbt/jaffle_shop/models/staging/sources.yml diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index 4f1fae227..48caaf97f 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -3,9 +3,11 @@ from typing import Any from airflow.io.path import ObjectStoragePath +from airflow.providers.google.cloud.hooks.bigquery import BigQueryHook from airflow.providers.google.cloud.operators.bigquery import BigQueryInsertJobOperator from airflow.utils.context import Context +from cosmos.exceptions import CosmosValueError from cosmos.operators.local import ( DbtBuildLocalOperator, DbtCompileLocalOperator, @@ -47,25 +49,31 @@ class DbtSourceAirflowAsyncOperator(DbtSourceLocalOperator): pass -class DbtRunAirflowAsyncOperator(BigQueryInsertJobOperator): - def __init__(self, *args, **kwargs): - self.configuration = {} - self.job_id = kwargs.get("job_id", {}) or "" - self.impersonation_chain = kwargs.get("impersonation_chain", {}) or "" - self.project_id = kwargs.get("project_id", {}) or "" - +class DbtRunAirflowAsyncOperator(BigQueryInsertJobOperator): # + def __init__(self, *args, full_refresh: bool = False, **kwargs): + # dbt task param self.profile_config = kwargs.get("profile_config") self.project_dir = kwargs.get("project_dir") + self.profile_type = self.profile_config.get_profile_type() + self.full_refresh = full_refresh - self.async_op_args = kwargs.get("async_op_args", {}) + # airflow task param + self.async_op_args = kwargs.pop("async_op_args", {}) + self.configuration = {} + self.job_id = self.async_op_args.get("job_id", "") + self.impersonation_chain = self.async_op_args.get("impersonation_chain", "") + self.gcp_project = self.async_op_args.get("project_id", "astronomer-dag-authoring") + self.gcp_conn_id = self.profile_config.profile_mapping.conn_id + self.dataset = self.async_op_args.get("dataset", "my_dataset") + self.location = self.async_op_args.get("location", "US") self.async_op_args["deferrable"] = True + self.reattach_states: set[str] = self.async_op_args.get("reattach_states") or set() + super().__init__(*args, configuration=self.configuration, task_id=kwargs.get("task_id"), **self.async_op_args) - self.profile_type = self.profile_config.get_profile_type() + if self.profile_type not in _SUPPORTED_DATABASES: raise f"Async run are only supported: {_SUPPORTED_DATABASES}" - self.reattach_states: set[str] = self.async_op_args.get("reattach_states") or set() - def get_remote_sql(self): project_name = str(self.project_dir).split("/")[-1] model_name: str = self.task_id.split(".")[0] @@ -79,17 +87,39 @@ def get_remote_sql(self): with object_storage_path.open() as fp: return fp.read() - def execute(self, context: Context) -> Any | None: - sql = self.get_remote_sql() - print("sql: ", sql) + def drop_table_sql(self): + model_name = self.task_id.split(".")[0] + sql = f"DROP TABLE IF EXISTS {self.gcp_project}.{self.dataset}.{model_name};" + hook = BigQueryHook( + gcp_conn_id=self.gcp_conn_id, + impersonation_chain=self.impersonation_chain, + ) self.configuration = { "query": { "query": sql, "useLegacySql": False, } } - print("async_op_args: ", self.async_op_args) - super().execute(context) + hook.insert_job(configuration=self.configuration, location=self.location, project_id=self.gcp_project) + + def execute(self, context: Context) -> Any | None: + if not self.full_refresh: + raise CosmosValueError("The async execution only supported for full_refresh") + else: + self.drop_table_sql() + + sql = self.get_remote_sql() + model_name = self.task_id.split(".")[0] + # prefix explicit create command to create table + sql = f"CREATE TABLE {self.gcp_project}.{self.dataset}.{model_name} AS {sql}" + + self.configuration = { + "query": { + "query": sql, + "useLegacySql": False, + } + } + super().execute(context) class DbtTestAirflowAsyncOperator(DbtTestLocalOperator): diff --git a/dev/dags/dbt/jaffle_shop/models/staging/sources.yml b/dev/dags/dbt/jaffle_shop/models/staging/sources.yml deleted file mode 100644 index a3139b585..000000000 --- a/dev/dags/dbt/jaffle_shop/models/staging/sources.yml +++ /dev/null @@ -1,31 +0,0 @@ - -version: 2 - -sources: - - name: postgres_db - database: "{{ env_var('POSTGRES_DB') }}" - schema: "{{ env_var('POSTGRES_SCHEMA') }}" - tables: - - name: raw_customers - columns: - - name: id - tests: - - unique - - not_null - - name: raw_payments - columns: - - name: id - tests: - - unique - - not_null - - name: raw_orders - columns: - - name: id - tests: - - unique - - not_null - freshness: - warn_after: - count: 3650 - period: day - loaded_at_field: CAST(order_date AS TIMESTAMP) From 1278847d4cea546bc74861f18b62bfdfa3b2369a Mon Sep 17 00:00:00 2001 From: "pre-commit-ci[bot]" <66853113+pre-commit-ci[bot]@users.noreply.github.com> Date: Mon, 30 Sep 2024 14:28:44 +0000 Subject: [PATCH 10/49] =?UTF-8?q?=F0=9F=8E=A8=20[pre-commit.ci]=20Auto=20f?= =?UTF-8?q?ormat=20from=20pre-commit.com=20hooks?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- cosmos/operators/local.py | 1 - 1 file changed, 1 deletion(-) diff --git a/cosmos/operators/local.py b/cosmos/operators/local.py index ad320d5fc..ee0754f75 100644 --- a/cosmos/operators/local.py +++ b/cosmos/operators/local.py @@ -291,7 +291,6 @@ def _configure_remote_target_path() -> tuple[Path, str] | tuple[None, None]: return _configured_target_path, remote_conn_id - def upload_compiled_sql(self, tmp_project_dir: str, context: Context) -> None: """ Uploads the compiled SQL files from the dbt compile output to the remote store. From b3d6cf327f401d9dd402e2698e51155ec21da6de Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Mon, 30 Sep 2024 21:53:09 +0530 Subject: [PATCH 11/49] Use dbt node's filepath to construct remote path to fetch compiled SQL from --- cosmos/operators/airflow_async.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index 48caaf97f..494dbbe27 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -54,6 +54,7 @@ def __init__(self, *args, full_refresh: bool = False, **kwargs): # dbt task param self.profile_config = kwargs.get("profile_config") self.project_dir = kwargs.get("project_dir") + self.file_path = kwargs.get("extra_context", {}).get("dbt_node_config", {}).get("file_path") self.profile_type = self.profile_config.get_profile_type() self.full_refresh = full_refresh @@ -75,12 +76,11 @@ def __init__(self, *args, full_refresh: bool = False, **kwargs): raise f"Async run are only supported: {_SUPPORTED_DATABASES}" def get_remote_sql(self): - project_name = str(self.project_dir).split("/")[-1] - model_name: str = self.task_id.split(".")[0] - if model_name.startswith("stg_"): - remote_model_path = f"{remote_target_path}/{self.dag_id}/{project_name}/models/staging/{model_name}.sql" - else: - remote_model_path = f"{remote_target_path}/{self.dag_id}/{project_name}/models/{model_name}.sql" + if not self.file_path or not self.project_dir: + raise CosmosValueError("file_path and project_dir are required to be set on the task for async execution") + project_dir_parent = str(self.project_dir.parent) + relative_file_path = str(self.file_path).replace(project_dir_parent, "").lstrip("/") + remote_model_path = f"{str(remote_target_path).rstrip('/')}/{self.dag_id}/{relative_file_path}" print("remote_model_path: ", remote_model_path) object_storage_path = ObjectStoragePath(remote_model_path, conn_id=remote_target_path_conn_id) From 9ca5e8545bb417b02265c247f51bcd1411114e7f Mon Sep 17 00:00:00 2001 From: "pre-commit-ci[bot]" <66853113+pre-commit-ci[bot]@users.noreply.github.com> Date: Mon, 30 Sep 2024 22:22:55 +0000 Subject: [PATCH 12/49] =?UTF-8?q?=F0=9F=8E=A8=20[pre-commit.ci]=20Auto=20f?= =?UTF-8?q?ormat=20from=20pre-commit.com=20hooks?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- cosmos/airflow/graph.py | 1 - cosmos/operators/airflow_async.py | 1 - tests/airflow/test_graph.py | 1 - 3 files changed, 3 deletions(-) diff --git a/cosmos/airflow/graph.py b/cosmos/airflow/graph.py index faad1f838..c94fd0246 100644 --- a/cosmos/airflow/graph.py +++ b/cosmos/airflow/graph.py @@ -21,7 +21,6 @@ from cosmos.core.graph.entities import Task as TaskMetadata from cosmos.dbt.graph import DbtNode from cosmos.log import get_logger -from cosmos.settings import dbt_compile_task_id logger = get_logger(__name__) diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index 11eb6bec7..24e8111ae 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -126,4 +126,3 @@ class DbtRunOperationAirflowAsyncOperator(DbtRunOperationLocalOperator): class DbtCompileAirflowAsyncOperator(DbtCompileLocalOperator): pass - diff --git a/tests/airflow/test_graph.py b/tests/airflow/test_graph.py index e902c915d..6fc7cdc0a 100644 --- a/tests/airflow/test_graph.py +++ b/tests/airflow/test_graph.py @@ -31,7 +31,6 @@ from cosmos.converter import airflow_kwargs from cosmos.dbt.graph import DbtNode from cosmos.profiles import PostgresUserPasswordProfileMapping -from cosmos.settings import dbt_compile_task_id SAMPLE_PROJ_PATH = Path("/home/user/path/dbt-proj/") SOURCE_RENDERING_BEHAVIOR = SourceRenderingBehavior(os.getenv("SOURCE_RENDERING_BEHAVIOR", "none")) From 99bf7c0fdf3d8b80673b3deb7d4fdf081c2a8566 Mon Sep 17 00:00:00 2001 From: Tatiana Al-Chueyr Date: Tue, 1 Oct 2024 00:31:42 +0100 Subject: [PATCH 13/49] Fix unittests --- cosmos/config.py | 10 +++++---- cosmos/operators/airflow_async.py | 28 +++++++++++++---------- tests/operators/test_airflow_async.py | 32 +++++++-------------------- tests/operators/test_local.py | 2 +- 4 files changed, 31 insertions(+), 41 deletions(-) diff --git a/cosmos/config.py b/cosmos/config.py index b92a25a20..77707830c 100644 --- a/cosmos/config.py +++ b/cosmos/config.py @@ -287,9 +287,9 @@ def validate_profiles_yml(self) -> None: if self.profiles_yml_filepath and not Path(self.profiles_yml_filepath).exists(): raise CosmosValueError(f"The file {self.profiles_yml_filepath} does not exist.") - def get_profile_type(self): - if self.profile_mapping.dbt_profile_type: - return self.profile_mapping.dbt_profile_type + def get_profile_type(self) -> str: + if self.profile_mapping is not None and hasattr(self.profile_mapping, "dbt_profile_type"): + return str(self.profile_mapping.dbt_profile_type) profile_path = self._get_profile_path() @@ -298,7 +298,9 @@ def get_profile_type(self): profile = profiles[self.profile_name] target_type = profile["outputs"][self.target_name]["type"] - return target_type + return str(target_type) + + return "undefined" def _get_profile_path(self, use_mock_values: bool = False) -> Path: """ diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index 24e8111ae..e7bc9b01e 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -2,11 +2,11 @@ from typing import Any -from airflow.io.path import ObjectStoragePath from airflow.providers.google.cloud.hooks.bigquery import BigQueryHook from airflow.providers.google.cloud.operators.bigquery import BigQueryInsertJobOperator from airflow.utils.context import Context +from cosmos import settings from cosmos.exceptions import CosmosValueError from cosmos.operators.local import ( DbtBuildLocalOperator, @@ -43,22 +43,22 @@ class DbtSourceAirflowAsyncOperator(DbtSourceLocalOperator): pass -class DbtRunAirflowAsyncOperator(BigQueryInsertJobOperator): # - def __init__(self, *args, full_refresh: bool = False, **kwargs): +class DbtRunAirflowAsyncOperator(BigQueryInsertJobOperator): # type: ignore + def __init__(self, *args, full_refresh: bool = False, **kwargs): # type: ignore # dbt task param self.profile_config = kwargs.get("profile_config") self.project_dir = kwargs.get("project_dir") self.file_path = kwargs.get("extra_context", {}).get("dbt_node_config", {}).get("file_path") - self.profile_type = self.profile_config.get_profile_type() + self.profile_type: str = self.profile_config.get_profile_type() # type: ignore self.full_refresh = full_refresh # airflow task param self.async_op_args = kwargs.pop("async_op_args", {}) - self.configuration = {} + self.configuration: dict[str, object] = {} self.job_id = self.async_op_args.get("job_id", "") self.impersonation_chain = self.async_op_args.get("impersonation_chain", "") self.gcp_project = self.async_op_args.get("project_id", "astronomer-dag-authoring") - self.gcp_conn_id = self.profile_config.profile_mapping.conn_id + self.gcp_conn_id = self.profile_config.profile_mapping.conn_id # type: ignore self.dataset = self.async_op_args.get("dataset", "my_dataset") self.location = self.async_op_args.get("location", "US") self.async_op_args["deferrable"] = True @@ -67,9 +67,13 @@ def __init__(self, *args, full_refresh: bool = False, **kwargs): super().__init__(*args, configuration=self.configuration, task_id=kwargs.get("task_id"), **self.async_op_args) if self.profile_type not in _SUPPORTED_DATABASES: - raise f"Async run are only supported: {_SUPPORTED_DATABASES}" + raise CosmosValueError(f"Async run are only supported: {_SUPPORTED_DATABASES}") + + def get_remote_sql(self) -> str: + if not settings.AIRFLOW_IO_AVAILABLE: + raise CosmosValueError(f"Cosmos async support is only available starting in Airflow 2.8 or later.") + from airflow.io.path import ObjectStoragePath - def get_remote_sql(self): if not self.file_path or not self.project_dir: raise CosmosValueError("file_path and project_dir are required to be set on the task for async execution") project_dir_parent = str(self.project_dir.parent) @@ -78,10 +82,10 @@ def get_remote_sql(self): print("remote_model_path: ", remote_model_path) object_storage_path = ObjectStoragePath(remote_model_path, conn_id=remote_target_path_conn_id) - with object_storage_path.open() as fp: - return fp.read() + with object_storage_path.open() as fp: # type: ignore + return fp.read() # type: ignore - def drop_table_sql(self): + def drop_table_sql(self) -> None: model_name = self.task_id.split(".")[0] sql = f"DROP TABLE IF EXISTS {self.gcp_project}.{self.dataset}.{model_name};" hook = BigQueryHook( @@ -113,7 +117,7 @@ def execute(self, context: Context) -> Any | None: "useLegacySql": False, } } - super().execute(context) + return super().execute(context) class DbtTestAirflowAsyncOperator(DbtTestLocalOperator): diff --git a/tests/operators/test_airflow_async.py b/tests/operators/test_airflow_async.py index fc085c7d0..ef58a4d48 100644 --- a/tests/operators/test_airflow_async.py +++ b/tests/operators/test_airflow_async.py @@ -1,10 +1,10 @@ +import pytest +from airflow import __version__ as airflow_version +from packaging import version + from cosmos.operators.airflow_async import ( DbtBuildAirflowAsyncOperator, DbtCompileAirflowAsyncOperator, - DbtDocsAirflowAsyncOperator, - DbtDocsAzureStorageAirflowAsyncOperator, - DbtDocsGCSAirflowAsyncOperator, - DbtDocsS3AirflowAsyncOperator, DbtLSAirflowAsyncOperator, DbtRunAirflowAsyncOperator, DbtRunOperationAirflowAsyncOperator, @@ -16,10 +16,6 @@ from cosmos.operators.local import ( DbtBuildLocalOperator, DbtCompileLocalOperator, - DbtDocsAzureStorageLocalOperator, - DbtDocsGCSLocalOperator, - DbtDocsLocalOperator, - DbtDocsS3LocalOperator, DbtLSLocalOperator, DbtRunLocalOperator, DbtRunOperationLocalOperator, @@ -50,6 +46,10 @@ def test_dbt_source_airflow_async_operator_inheritance(): assert issubclass(DbtSourceAirflowAsyncOperator, DbtSourceLocalOperator) +@pytest.mark.skipif( + version.parse(airflow_version) < version.parse("2.8"), + reason="Cosmos Async operators only work with Airflow 2.8 onwards.", +) def test_dbt_run_airflow_async_operator_inheritance(): assert issubclass(DbtRunAirflowAsyncOperator, DbtRunLocalOperator) @@ -62,21 +62,5 @@ def test_dbt_run_operation_airflow_async_operator_inheritance(): assert issubclass(DbtRunOperationAirflowAsyncOperator, DbtRunOperationLocalOperator) -def test_dbt_docs_airflow_async_operator_inheritance(): - assert issubclass(DbtDocsAirflowAsyncOperator, DbtDocsLocalOperator) - - -def test_dbt_docs_s3_airflow_async_operator_inheritance(): - assert issubclass(DbtDocsS3AirflowAsyncOperator, DbtDocsS3LocalOperator) - - -def test_dbt_docs_azure_storage_airflow_async_operator_inheritance(): - assert issubclass(DbtDocsAzureStorageAirflowAsyncOperator, DbtDocsAzureStorageLocalOperator) - - -def test_dbt_docs_gcs_airflow_async_operator_inheritance(): - assert issubclass(DbtDocsGCSAirflowAsyncOperator, DbtDocsGCSLocalOperator) - - def test_dbt_compile_airflow_async_operator_inheritance(): assert issubclass(DbtCompileAirflowAsyncOperator, DbtCompileLocalOperator) diff --git a/tests/operators/test_local.py b/tests/operators/test_local.py index c7615225f..bfd1408d9 100644 --- a/tests/operators/test_local.py +++ b/tests/operators/test_local.py @@ -1147,7 +1147,7 @@ def test_dbt_compile_local_operator_initialisation(): @patch("cosmos.operators.local.remote_target_path", new="s3://some-bucket/target") -@patch("cosmos.operators.local.AIRFLOW_IO_AVAILABLE", new=False) +@patch("cosmos.settings.AIRFLOW_IO_AVAILABLE", new=False) def test_configure_remote_target_path_object_storage_unavailable_on_earlier_airflow_versions(): operator = DbtCompileLocalOperator( task_id="fake-task", From 3aaaf9e58741b8f92a1f82b5152876ae0ae0316a Mon Sep 17 00:00:00 2001 From: Tatiana Al-Chueyr Date: Tue, 1 Oct 2024 00:32:30 +0100 Subject: [PATCH 14/49] Improve code --- cosmos/config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cosmos/config.py b/cosmos/config.py index 77707830c..ccda2c432 100644 --- a/cosmos/config.py +++ b/cosmos/config.py @@ -288,7 +288,7 @@ def validate_profiles_yml(self) -> None: raise CosmosValueError(f"The file {self.profiles_yml_filepath} does not exist.") def get_profile_type(self) -> str: - if self.profile_mapping is not None and hasattr(self.profile_mapping, "dbt_profile_type"): + if isinstance(self.profile_mapping, BaseProfileMapping): return str(self.profile_mapping.dbt_profile_type) profile_path = self._get_profile_path() From 43158be43079e872feb4abb4f21e6309bbe56982 Mon Sep 17 00:00:00 2001 From: Tatiana Al-Chueyr Date: Tue, 1 Oct 2024 10:00:02 +0100 Subject: [PATCH 15/49] Working with deferrable=False, not working with deferrable=True --- .gitignore | 3 - cosmos/operators/airflow_async.py | 18 +- cosmos/operators/local.py | 3 +- dev/dags/dbt/original_jaffle_shop/.gitignore | 5 + dev/dags/dbt/original_jaffle_shop/LICENSE | 201 ++++++++++++++++++ dev/dags/dbt/original_jaffle_shop/README.md | 11 + .../dbt/original_jaffle_shop/dbt_project.yml | 26 +++ .../macros/drop_table.sql | 6 + .../original_jaffle_shop/models/customers.sql | 69 ++++++ .../dbt/original_jaffle_shop/models/docs.md | 14 ++ .../original_jaffle_shop/models/orders.sql | 56 +++++ .../original_jaffle_shop/models/overview.md | 11 + .../original_jaffle_shop/models/schema.yml | 82 +++++++ .../models/staging/schema.yml | 31 +++ .../models/staging/stg_customers.sql | 22 ++ .../models/staging/stg_orders.sql | 23 ++ .../models/staging/stg_payments.sql | 25 +++ .../seeds/raw_customers.csv | 101 +++++++++ .../original_jaffle_shop/seeds/raw_orders.csv | 100 +++++++++ .../seeds/raw_payments.csv | 114 ++++++++++ dev/dags/simple_dag_async.py | 19 +- 21 files changed, 916 insertions(+), 24 deletions(-) create mode 100644 dev/dags/dbt/original_jaffle_shop/.gitignore create mode 100644 dev/dags/dbt/original_jaffle_shop/LICENSE create mode 100644 dev/dags/dbt/original_jaffle_shop/README.md create mode 100644 dev/dags/dbt/original_jaffle_shop/dbt_project.yml create mode 100644 dev/dags/dbt/original_jaffle_shop/macros/drop_table.sql create mode 100644 dev/dags/dbt/original_jaffle_shop/models/customers.sql create mode 100644 dev/dags/dbt/original_jaffle_shop/models/docs.md create mode 100644 dev/dags/dbt/original_jaffle_shop/models/orders.sql create mode 100644 dev/dags/dbt/original_jaffle_shop/models/overview.md create mode 100644 dev/dags/dbt/original_jaffle_shop/models/schema.yml create mode 100644 dev/dags/dbt/original_jaffle_shop/models/staging/schema.yml create mode 100644 dev/dags/dbt/original_jaffle_shop/models/staging/stg_customers.sql create mode 100644 dev/dags/dbt/original_jaffle_shop/models/staging/stg_orders.sql create mode 100644 dev/dags/dbt/original_jaffle_shop/models/staging/stg_payments.sql create mode 100644 dev/dags/dbt/original_jaffle_shop/seeds/raw_customers.csv create mode 100644 dev/dags/dbt/original_jaffle_shop/seeds/raw_orders.csv create mode 100644 dev/dags/dbt/original_jaffle_shop/seeds/raw_payments.csv diff --git a/.gitignore b/.gitignore index 6f6111510..5991c231c 100644 --- a/.gitignore +++ b/.gitignore @@ -160,6 +160,3 @@ webserver_config.py # VI *.sw[a-z] - -# Ignore possibly created symlink to `dev/dags` for running `airflow dags test` command. -dags diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index e7bc9b01e..ee27e2aff 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -53,18 +53,14 @@ def __init__(self, *args, full_refresh: bool = False, **kwargs): # type: ignore self.full_refresh = full_refresh # airflow task param - self.async_op_args = kwargs.pop("async_op_args", {}) self.configuration: dict[str, object] = {} - self.job_id = self.async_op_args.get("job_id", "") - self.impersonation_chain = self.async_op_args.get("impersonation_chain", "") - self.gcp_project = self.async_op_args.get("project_id", "astronomer-dag-authoring") self.gcp_conn_id = self.profile_config.profile_mapping.conn_id # type: ignore - self.dataset = self.async_op_args.get("dataset", "my_dataset") - self.location = self.async_op_args.get("location", "US") - self.async_op_args["deferrable"] = True - self.reattach_states: set[str] = self.async_op_args.get("reattach_states") or set() - - super().__init__(*args, configuration=self.configuration, task_id=kwargs.get("task_id"), **self.async_op_args) + if not self.profile_config or not self.profile_config.profile_mapping: + raise CosmosValueError(f"Cosmos async support is only available starting in Airflow 2.8 or later.") + profile = self.profile_config.profile_mapping.profile # type: ignore + self.gcp_project = profile["project"] + self.dataset = profile["dataset"] + super().__init__(*args, configuration=self.configuration, task_id=kwargs["task_id"], deferrable=True) if self.profile_type not in _SUPPORTED_DATABASES: raise CosmosValueError(f"Async run are only supported: {_SUPPORTED_DATABASES}") @@ -78,7 +74,7 @@ def get_remote_sql(self) -> str: raise CosmosValueError("file_path and project_dir are required to be set on the task for async execution") project_dir_parent = str(self.project_dir.parent) relative_file_path = str(self.file_path).replace(project_dir_parent, "").lstrip("/") - remote_model_path = f"{str(remote_target_path).rstrip('/')}/{self.dag_id}/{relative_file_path}" + remote_model_path = f"{str(remote_target_path).rstrip('/')}/{self.dag_id}/compiled/{relative_file_path}" print("remote_model_path: ", remote_model_path) object_storage_path = ObjectStoragePath(remote_model_path, conn_id=remote_target_path_conn_id) diff --git a/cosmos/operators/local.py b/cosmos/operators/local.py index 3de3ce076..b40cb411c 100644 --- a/cosmos/operators/local.py +++ b/cosmos/operators/local.py @@ -352,7 +352,6 @@ def upload_compiled_sql(self, tmp_project_dir: str, context: Context) -> None: source_compiled_dir = Path(tmp_project_dir) / "target" / "compiled" files = [str(file) for file in source_compiled_dir.rglob("*") if file.is_file()] - for file_path in files: dest_file_path = self._construct_dest_file_path(dest_target_dir, file_path, source_compiled_dir, context) dest_object_storage_path = ObjectStoragePath(dest_file_path, conn_id=dest_conn_id) @@ -1013,5 +1012,5 @@ def __init__(self, **kwargs: str) -> None: class DbtCompileLocalOperator(DbtCompileMixin, DbtLocalBaseOperator): def __init__(self, *args: Any, **kwargs: Any) -> None: - kwargs["should_upload_compiled_sql"] = True + kwargs["should_upload_compiled_sql"] = False super().__init__(*args, **kwargs) diff --git a/dev/dags/dbt/original_jaffle_shop/.gitignore b/dev/dags/dbt/original_jaffle_shop/.gitignore new file mode 100644 index 000000000..45d294b9a --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/.gitignore @@ -0,0 +1,5 @@ + +target/ +dbt_packages/ +logs/ +!target/manifest.json diff --git a/dev/dags/dbt/original_jaffle_shop/LICENSE b/dev/dags/dbt/original_jaffle_shop/LICENSE new file mode 100644 index 000000000..8dada3eda --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright {yyyy} {name of copyright owner} + + Licensed 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. diff --git a/dev/dags/dbt/original_jaffle_shop/README.md b/dev/dags/dbt/original_jaffle_shop/README.md new file mode 100644 index 000000000..d4ce46446 --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/README.md @@ -0,0 +1,11 @@ +## `jaffle_shop` + +`jaffle_shop` is a fictional ecommerce store. This dbt project transforms raw data from an app database into a customers and orders model ready for analytics. + +See [dbt's documentation](https://github.com/dbt-labs/jaffle_shop) for more info. + +### Modifications + +This project has been modified from the original to highlight some of the features of Cosmos. Namely: + +- tags have been added to the models diff --git a/dev/dags/dbt/original_jaffle_shop/dbt_project.yml b/dev/dags/dbt/original_jaffle_shop/dbt_project.yml new file mode 100644 index 000000000..42767c5ea --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/dbt_project.yml @@ -0,0 +1,26 @@ +name: 'original_jaffle_shop' + +config-version: 2 +version: '0.1' + +profile: 'jaffle_shop' + +model-paths: ["models"] +seed-paths: ["seeds"] +test-paths: ["tests"] +analysis-paths: ["analysis"] +macro-paths: ["macros"] + +target-path: "target" +clean-targets: + - "target" + - "dbt_modules" + - "logs" + +require-dbt-version: [">=1.0.0", "<2.0.0"] + +models: + jaffle_shop: + materialized: table + staging: + materialized: view diff --git a/dev/dags/dbt/original_jaffle_shop/macros/drop_table.sql b/dev/dags/dbt/original_jaffle_shop/macros/drop_table.sql new file mode 100644 index 000000000..37a8b21d7 --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/macros/drop_table.sql @@ -0,0 +1,6 @@ +{%- macro drop_table_by_name(table_name) -%} + {%- set drop_query -%} + DROP TABLE IF EXISTS {{ target.schema }}.{{ table_name }} CASCADE + {%- endset -%} + {% do run_query(drop_query) %} +{%- endmacro -%} diff --git a/dev/dags/dbt/original_jaffle_shop/models/customers.sql b/dev/dags/dbt/original_jaffle_shop/models/customers.sql new file mode 100644 index 000000000..016a004fe --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/models/customers.sql @@ -0,0 +1,69 @@ +with customers as ( + + select * from {{ ref('stg_customers') }} + +), + +orders as ( + + select * from {{ ref('stg_orders') }} + +), + +payments as ( + + select * from {{ ref('stg_payments') }} + +), + +customer_orders as ( + + select + customer_id, + + min(order_date) as first_order, + max(order_date) as most_recent_order, + count(order_id) as number_of_orders + from orders + + group by customer_id + +), + +customer_payments as ( + + select + orders.customer_id, + sum(amount) as total_amount + + from payments + + left join orders on + payments.order_id = orders.order_id + + group by orders.customer_id + +), + +final as ( + + select + customers.customer_id, + customers.first_name, + customers.last_name, + customer_orders.first_order, + customer_orders.most_recent_order, + customer_orders.number_of_orders, + customer_payments.total_amount as customer_lifetime_value + + from customers + + left join customer_orders + on customers.customer_id = customer_orders.customer_id + + left join customer_payments + on customers.customer_id = customer_payments.customer_id + +) + +select * from final diff --git a/dev/dags/dbt/original_jaffle_shop/models/docs.md b/dev/dags/dbt/original_jaffle_shop/models/docs.md new file mode 100644 index 000000000..c6ae93be0 --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/models/docs.md @@ -0,0 +1,14 @@ +{% docs orders_status %} + +Orders can be one of the following statuses: + +| status | description | +|----------------|------------------------------------------------------------------------------------------------------------------------| +| placed | The order has been placed but has not yet left the warehouse | +| shipped | The order has ben shipped to the customer and is currently in transit | +| completed | The order has been received by the customer | +| return_pending | The customer has indicated that they would like to return the order, but it has not yet been received at the warehouse | +| returned | The order has been returned by the customer and received at the warehouse | + + +{% enddocs %} diff --git a/dev/dags/dbt/original_jaffle_shop/models/orders.sql b/dev/dags/dbt/original_jaffle_shop/models/orders.sql new file mode 100644 index 000000000..cbb293491 --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/models/orders.sql @@ -0,0 +1,56 @@ +{% set payment_methods = ['credit_card', 'coupon', 'bank_transfer', 'gift_card'] %} + +with orders as ( + + select * from {{ ref('stg_orders') }} + +), + +payments as ( + + select * from {{ ref('stg_payments') }} + +), + +order_payments as ( + + select + order_id, + + {% for payment_method in payment_methods -%} + sum(case when payment_method = '{{ payment_method }}' then amount else 0 end) as {{ payment_method }}_amount, + {% endfor -%} + + sum(amount) as total_amount + + from payments + + group by order_id + +), + +final as ( + + select + orders.order_id, + orders.customer_id, + orders.order_date, + orders.status, + + {% for payment_method in payment_methods -%} + + order_payments.{{ payment_method }}_amount, + + {% endfor -%} + + order_payments.total_amount as amount + + from orders + + + left join order_payments + on orders.order_id = order_payments.order_id + +) + +select * from final diff --git a/dev/dags/dbt/original_jaffle_shop/models/overview.md b/dev/dags/dbt/original_jaffle_shop/models/overview.md new file mode 100644 index 000000000..0544c42b1 --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/models/overview.md @@ -0,0 +1,11 @@ +{% docs __overview__ %} + +## Data Documentation for Jaffle Shop + +`jaffle_shop` is a fictional ecommerce store. + +This [dbt](https://www.getdbt.com/) project is for testing out code. + +The source code can be found [here](https://github.com/clrcrl/jaffle_shop). + +{% enddocs %} diff --git a/dev/dags/dbt/original_jaffle_shop/models/schema.yml b/dev/dags/dbt/original_jaffle_shop/models/schema.yml new file mode 100644 index 000000000..381349cfd --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/models/schema.yml @@ -0,0 +1,82 @@ +version: 2 + +models: + - name: customers + description: This table has basic information about a customer, as well as some derived facts based on a customer's orders + + columns: + - name: customer_id + description: This is a unique identifier for a customer + tests: + - unique + - not_null + + - name: first_name + description: Customer's first name. PII. + + - name: last_name + description: Customer's last name. PII. + + - name: first_order + description: Date (UTC) of a customer's first order + + - name: most_recent_order + description: Date (UTC) of a customer's most recent order + + - name: number_of_orders + description: Count of the number of orders a customer has placed + + - name: total_order_amount + description: Total value (AUD) of a customer's orders + + - name: orders + description: This table has basic information about orders, as well as some derived facts based on payments + + columns: + - name: order_id + tests: + - unique + - not_null + description: This is a unique identifier for an order + + - name: customer_id + description: Foreign key to the customers table + tests: + - not_null + - relationships: + to: ref('customers') + field: customer_id + + - name: order_date + description: Date (UTC) that the order was placed + + - name: status + description: '{{ doc("orders_status") }}' + tests: + - accepted_values: + values: ['placed', 'shipped', 'completed', 'return_pending', 'returned'] + + - name: amount + description: Total amount (AUD) of the order + tests: + - not_null + + - name: credit_card_amount + description: Amount of the order (AUD) paid for by credit card + tests: + - not_null + + - name: coupon_amount + description: Amount of the order (AUD) paid for by coupon + tests: + - not_null + + - name: bank_transfer_amount + description: Amount of the order (AUD) paid for by bank transfer + tests: + - not_null + + - name: gift_card_amount + description: Amount of the order (AUD) paid for by gift card + tests: + - not_null diff --git a/dev/dags/dbt/original_jaffle_shop/models/staging/schema.yml b/dev/dags/dbt/original_jaffle_shop/models/staging/schema.yml new file mode 100644 index 000000000..c207e4cf5 --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/models/staging/schema.yml @@ -0,0 +1,31 @@ +version: 2 + +models: + - name: stg_customers + columns: + - name: customer_id + tests: + - unique + - not_null + + - name: stg_orders + columns: + - name: order_id + tests: + - unique + - not_null + - name: status + tests: + - accepted_values: + values: ['placed', 'shipped', 'completed', 'return_pending', 'returned'] + + - name: stg_payments + columns: + - name: payment_id + tests: + - unique + - not_null + - name: payment_method + tests: + - accepted_values: + values: ['credit_card', 'coupon', 'bank_transfer', 'gift_card'] diff --git a/dev/dags/dbt/original_jaffle_shop/models/staging/stg_customers.sql b/dev/dags/dbt/original_jaffle_shop/models/staging/stg_customers.sql new file mode 100644 index 000000000..cad047269 --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/models/staging/stg_customers.sql @@ -0,0 +1,22 @@ +with source as ( + + {#- + Normally we would select from the table here, but we are using seeds to load + our data in this project + #} + select * from {{ ref('raw_customers') }} + +), + +renamed as ( + + select + id as customer_id, + first_name, + last_name + + from source + +) + +select * from renamed diff --git a/dev/dags/dbt/original_jaffle_shop/models/staging/stg_orders.sql b/dev/dags/dbt/original_jaffle_shop/models/staging/stg_orders.sql new file mode 100644 index 000000000..a654dcb94 --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/models/staging/stg_orders.sql @@ -0,0 +1,23 @@ +with source as ( + + {#- + Normally we would select from the table here, but we are using seeds to load + our data in this project + #} + select * from {{ ref('raw_orders') }} + +), + +renamed as ( + + select + id as order_id, + user_id as customer_id, + order_date, + status + + from source + +) + +select * from renamed diff --git a/dev/dags/dbt/original_jaffle_shop/models/staging/stg_payments.sql b/dev/dags/dbt/original_jaffle_shop/models/staging/stg_payments.sql new file mode 100644 index 000000000..f718596ad --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/models/staging/stg_payments.sql @@ -0,0 +1,25 @@ +with source as ( + + {#- + Normally we would select from the table here, but we are using seeds to load + our data in this project + #} + select * from {{ ref('raw_payments') }} + +), + +renamed as ( + + select + id as payment_id, + order_id, + payment_method, + + -- `amount` is currently stored in cents, so we convert it to dollars + amount / 100 as amount + + from source + +) + +select * from renamed diff --git a/dev/dags/dbt/original_jaffle_shop/seeds/raw_customers.csv b/dev/dags/dbt/original_jaffle_shop/seeds/raw_customers.csv new file mode 100644 index 000000000..b3e6747d6 --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/seeds/raw_customers.csv @@ -0,0 +1,101 @@ +id,first_name,last_name +1,Michael,P. +2,Shawn,M. +3,Kathleen,P. +4,Jimmy,C. +5,Katherine,R. +6,Sarah,R. +7,Martin,M. +8,Frank,R. +9,Jennifer,F. +10,Henry,W. +11,Fred,S. +12,Amy,D. +13,Kathleen,M. +14,Steve,F. +15,Teresa,H. +16,Amanda,H. +17,Kimberly,R. +18,Johnny,K. +19,Virginia,F. +20,Anna,A. +21,Willie,H. +22,Sean,H. +23,Mildred,A. +24,David,G. +25,Victor,H. +26,Aaron,R. +27,Benjamin,B. +28,Lisa,W. +29,Benjamin,K. +30,Christina,W. +31,Jane,G. +32,Thomas,O. +33,Katherine,M. +34,Jennifer,S. +35,Sara,T. +36,Harold,O. +37,Shirley,J. +38,Dennis,J. +39,Louise,W. +40,Maria,A. +41,Gloria,C. +42,Diana,S. +43,Kelly,N. +44,Jane,R. +45,Scott,B. +46,Norma,C. +47,Marie,P. +48,Lillian,C. +49,Judy,N. +50,Billy,L. +51,Howard,R. +52,Laura,F. +53,Anne,B. +54,Rose,M. +55,Nicholas,R. +56,Joshua,K. +57,Paul,W. +58,Kathryn,K. +59,Adam,A. +60,Norma,W. +61,Timothy,R. +62,Elizabeth,P. +63,Edward,G. +64,David,C. +65,Brenda,W. +66,Adam,W. +67,Michael,H. +68,Jesse,E. +69,Janet,P. +70,Helen,F. +71,Gerald,C. +72,Kathryn,O. +73,Alan,B. +74,Harry,A. +75,Andrea,H. +76,Barbara,W. +77,Anne,W. +78,Harry,H. +79,Jack,R. +80,Phillip,H. +81,Shirley,H. +82,Arthur,D. +83,Virginia,R. +84,Christina,R. +85,Theresa,M. +86,Jason,C. +87,Phillip,B. +88,Adam,T. +89,Margaret,J. +90,Paul,P. +91,Todd,W. +92,Willie,O. +93,Frances,R. +94,Gregory,H. +95,Lisa,P. +96,Jacqueline,A. +97,Shirley,D. +98,Nicole,M. +99,Mary,G. +100,Jean,M. diff --git a/dev/dags/dbt/original_jaffle_shop/seeds/raw_orders.csv b/dev/dags/dbt/original_jaffle_shop/seeds/raw_orders.csv new file mode 100644 index 000000000..c4870621b --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/seeds/raw_orders.csv @@ -0,0 +1,100 @@ +id,user_id,order_date,status +1,1,2018-01-01,returned +2,3,2018-01-02,completed +3,94,2018-01-04,completed +4,50,2018-01-05,completed +5,64,2018-01-05,completed +6,54,2018-01-07,completed +7,88,2018-01-09,completed +8,2,2018-01-11,returned +9,53,2018-01-12,completed +10,7,2018-01-14,completed +11,99,2018-01-14,completed +12,59,2018-01-15,completed +13,84,2018-01-17,completed +14,40,2018-01-17,returned +15,25,2018-01-17,completed +16,39,2018-01-18,completed +17,71,2018-01-18,completed +18,64,2018-01-20,returned +19,54,2018-01-22,completed +20,20,2018-01-23,completed +21,71,2018-01-23,completed +22,86,2018-01-24,completed +23,22,2018-01-26,return_pending +24,3,2018-01-27,completed +25,51,2018-01-28,completed +26,32,2018-01-28,completed +27,94,2018-01-29,completed +28,8,2018-01-29,completed +29,57,2018-01-31,completed +30,69,2018-02-02,completed +31,16,2018-02-02,completed +32,28,2018-02-04,completed +33,42,2018-02-04,completed +34,38,2018-02-06,completed +35,80,2018-02-08,completed +36,85,2018-02-10,completed +37,1,2018-02-10,completed +38,51,2018-02-10,completed +39,26,2018-02-11,completed +40,33,2018-02-13,completed +41,99,2018-02-14,completed +42,92,2018-02-16,completed +43,31,2018-02-17,completed +44,66,2018-02-17,completed +45,22,2018-02-17,completed +46,6,2018-02-19,completed +47,50,2018-02-20,completed +48,27,2018-02-21,completed +49,35,2018-02-21,completed +50,51,2018-02-23,completed +51,71,2018-02-24,completed +52,54,2018-02-25,return_pending +53,34,2018-02-26,completed +54,54,2018-02-26,completed +55,18,2018-02-27,completed +56,79,2018-02-28,completed +57,93,2018-03-01,completed +58,22,2018-03-01,completed +59,30,2018-03-02,completed +60,12,2018-03-03,completed +61,63,2018-03-03,completed +62,57,2018-03-05,completed +63,70,2018-03-06,completed +64,13,2018-03-07,completed +65,26,2018-03-08,completed +66,36,2018-03-10,completed +67,79,2018-03-11,completed +68,53,2018-03-11,completed +69,3,2018-03-11,completed +70,8,2018-03-12,completed +71,42,2018-03-12,shipped +72,30,2018-03-14,shipped +73,19,2018-03-16,completed +74,9,2018-03-17,shipped +75,69,2018-03-18,completed +76,25,2018-03-20,completed +77,35,2018-03-21,shipped +78,90,2018-03-23,shipped +79,52,2018-03-23,shipped +80,11,2018-03-23,shipped +81,76,2018-03-23,shipped +82,46,2018-03-24,shipped +83,54,2018-03-24,shipped +84,70,2018-03-26,placed +85,47,2018-03-26,shipped +86,68,2018-03-26,placed +87,46,2018-03-27,placed +88,91,2018-03-27,shipped +89,21,2018-03-28,placed +90,66,2018-03-30,shipped +91,47,2018-03-31,placed +92,84,2018-04-02,placed +93,66,2018-04-03,placed +94,63,2018-04-03,placed +95,27,2018-04-04,placed +96,90,2018-04-06,placed +97,89,2018-04-07,placed +98,41,2018-04-07,placed +99,85,2018-04-09,placed diff --git a/dev/dags/dbt/original_jaffle_shop/seeds/raw_payments.csv b/dev/dags/dbt/original_jaffle_shop/seeds/raw_payments.csv new file mode 100644 index 000000000..a587baab5 --- /dev/null +++ b/dev/dags/dbt/original_jaffle_shop/seeds/raw_payments.csv @@ -0,0 +1,114 @@ +id,order_id,payment_method,amount +1,1,credit_card,1000 +2,2,credit_card,2000 +3,3,coupon,100 +4,4,coupon,2500 +5,5,bank_transfer,1700 +6,6,credit_card,600 +7,7,credit_card,1600 +8,8,credit_card,2300 +9,9,gift_card,2300 +10,9,bank_transfer,0 +11,10,bank_transfer,2600 +12,11,credit_card,2700 +13,12,credit_card,100 +14,13,credit_card,500 +15,13,bank_transfer,1400 +16,14,bank_transfer,300 +17,15,coupon,2200 +18,16,credit_card,1000 +19,17,bank_transfer,200 +20,18,credit_card,500 +21,18,credit_card,800 +22,19,gift_card,600 +23,20,bank_transfer,1500 +24,21,credit_card,1200 +25,22,bank_transfer,800 +26,23,gift_card,2300 +27,24,coupon,2600 +28,25,bank_transfer,2000 +29,25,credit_card,2200 +30,25,coupon,1600 +31,26,credit_card,3000 +32,27,credit_card,2300 +33,28,bank_transfer,1900 +34,29,bank_transfer,1200 +35,30,credit_card,1300 +36,31,credit_card,1200 +37,32,credit_card,300 +38,33,credit_card,2200 +39,34,bank_transfer,1500 +40,35,credit_card,2900 +41,36,bank_transfer,900 +42,37,credit_card,2300 +43,38,credit_card,1500 +44,39,bank_transfer,800 +45,40,credit_card,1400 +46,41,credit_card,1700 +47,42,coupon,1700 +48,43,gift_card,1800 +49,44,gift_card,1100 +50,45,bank_transfer,500 +51,46,bank_transfer,800 +52,47,credit_card,2200 +53,48,bank_transfer,300 +54,49,credit_card,600 +55,49,credit_card,900 +56,50,credit_card,2600 +57,51,credit_card,2900 +58,51,credit_card,100 +59,52,bank_transfer,1500 +60,53,credit_card,300 +61,54,credit_card,1800 +62,54,bank_transfer,1100 +63,55,credit_card,2900 +64,56,credit_card,400 +65,57,bank_transfer,200 +66,58,coupon,1800 +67,58,gift_card,600 +68,59,gift_card,2800 +69,60,credit_card,400 +70,61,bank_transfer,1600 +71,62,gift_card,1400 +72,63,credit_card,2900 +73,64,bank_transfer,2600 +74,65,credit_card,0 +75,66,credit_card,2800 +76,67,bank_transfer,400 +77,67,credit_card,1900 +78,68,credit_card,1600 +79,69,credit_card,1900 +80,70,credit_card,2600 +81,71,credit_card,500 +82,72,credit_card,2900 +83,73,bank_transfer,300 +84,74,credit_card,3000 +85,75,credit_card,1900 +86,76,coupon,200 +87,77,credit_card,0 +88,77,bank_transfer,1900 +89,78,bank_transfer,2600 +90,79,credit_card,1800 +91,79,credit_card,900 +92,80,gift_card,300 +93,81,coupon,200 +94,82,credit_card,800 +95,83,credit_card,100 +96,84,bank_transfer,2500 +97,85,bank_transfer,1700 +98,86,coupon,2300 +99,87,gift_card,3000 +100,87,credit_card,2600 +101,88,credit_card,2900 +102,89,bank_transfer,2200 +103,90,bank_transfer,200 +104,91,credit_card,1900 +105,92,bank_transfer,1500 +106,92,coupon,200 +107,93,gift_card,2600 +108,94,coupon,700 +109,95,coupon,2400 +110,96,gift_card,1700 +111,97,bank_transfer,1400 +112,98,bank_transfer,1000 +113,99,credit_card,2400 diff --git a/dev/dags/simple_dag_async.py b/dev/dags/simple_dag_async.py index 787461236..e80145215 100644 --- a/dev/dags/simple_dag_async.py +++ b/dev/dags/simple_dag_async.py @@ -2,8 +2,8 @@ from datetime import datetime from pathlib import Path -from cosmos import DbtDag, ExecutionConfig, ExecutionMode, ProfileConfig, ProjectConfig -from cosmos.profiles import PostgresUserPasswordProfileMapping +from cosmos import DbtDag, ExecutionConfig, ExecutionMode, ProfileConfig, ProjectConfig, RenderConfig +from cosmos.profiles import GoogleCloudServiceAccountFileProfileMapping DEFAULT_DBT_ROOT_PATH = Path(__file__).parent / "dbt" DBT_ROOT_PATH = Path(os.getenv("DBT_ROOT_PATH", DEFAULT_DBT_ROOT_PATH)) @@ -11,29 +11,32 @@ profile_config = ProfileConfig( profile_name="default", target_name="dev", - profile_mapping=PostgresUserPasswordProfileMapping( - conn_id="example_conn", - profile_args={"schema": "public"}, - disable_event_tracking=True, + profile_mapping=GoogleCloudServiceAccountFileProfileMapping( + conn_id="google_cloud_default", profile_args={"dataset": "release_17"} ), ) + # [START airflow_async_execution_mode_example] simple_dag_async = DbtDag( # dbt/cosmos-specific parameters project_config=ProjectConfig( - DBT_ROOT_PATH / "jaffle_shop", + DBT_ROOT_PATH / "original_jaffle_shop", ), profile_config=profile_config, execution_config=ExecutionConfig( execution_mode=ExecutionMode.AIRFLOW_ASYNC, ), + render_config=RenderConfig( + select=["path:models"], + # test_behavior=TestBehavior.NONE + ), # normal dag parameters schedule_interval=None, start_date=datetime(2023, 1, 1), catchup=False, dag_id="simple_dag_async", tags=["simple"], - operator_args={"install_deps": True}, + operator_args={"install_deps": True, "full_refresh": True}, ) # [END airflow_async_execution_mode_example] From 83b1010c1d535be822b5eaa905f491e7efa58ad7 Mon Sep 17 00:00:00 2001 From: Tatiana Al-Chueyr Date: Tue, 1 Oct 2024 10:00:12 +0100 Subject: [PATCH 16/49] Working with deferrable=False, not working with deferrable=True --- cosmos/operators/airflow_async.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index ee27e2aff..4089795de 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -57,7 +57,7 @@ def __init__(self, *args, full_refresh: bool = False, **kwargs): # type: ignore self.gcp_conn_id = self.profile_config.profile_mapping.conn_id # type: ignore if not self.profile_config or not self.profile_config.profile_mapping: raise CosmosValueError(f"Cosmos async support is only available starting in Airflow 2.8 or later.") - profile = self.profile_config.profile_mapping.profile # type: ignore + profile = self.profile_config.profile_mapping.profile self.gcp_project = profile["project"] self.dataset = profile["dataset"] super().__init__(*args, configuration=self.configuration, task_id=kwargs["task_id"], deferrable=True) From bd6657a29b111510fc34b2baf0bcc0d65ec0e5b9 Mon Sep 17 00:00:00 2001 From: Tatiana Al-Chueyr Date: Tue, 1 Oct 2024 11:09:05 +0100 Subject: [PATCH 17/49] Fix issue when using BQ deferrable operator - it requires location Otherwise, it raises: return func(self, *args, **kwargs) File "/Users/tati/Code/cosmos-clean/astronomer-cosmos/cosmos/operators/airflow_async.py", line 110, in execute self.drop_table_sql() File "/Users/tati/Code/cosmos-clean/astronomer-cosmos/cosmos/operators/airflow_async.py", line 104, in drop_table_sql hook.insert_job(configuration=self.configuration, location=self.location, project_id=self.gcp_project) File "/Users/tati/Code/cosmos-clean/astronomer-cosmos/venv-210/lib/python3.10/site-packages/airflow/providers/google/common/hooks/base_google.py", line 528, in inner_wrapper return func(self, *args, **kwargs) File "/Users/tati/Code/cosmos-clean/astronomer-cosmos/venv-210/lib/python3.10/site-packages/airflow/providers/google/cloud/hooks/bigquery.py", line 1676, in insert_job job_api_repr.result(timeout=timeout, retry=retry) File "/Users/tati/Code/cosmos-clean/astronomer-cosmos/venv-210/lib/python3.10/site-packages/google/cloud/bigquery/job/query.py", line 1590, in result do_get_result() File "/Users/tati/Code/cosmos-clean/astronomer-cosmos/venv-210/lib/python3.10/site-packages/google/cloud/bigquery/job/query.py", line 1579, in do_get_result super(QueryJob, self).result(retry=retry, timeout=timeout) File "/Users/tati/Code/cosmos-clean/astronomer-cosmos/venv-210/lib/python3.10/site-packages/google/cloud/bigquery/job/base.py", line 971, in result return super(_AsyncJob, self).result(timeout=timeout, **kwargs) File "/Users/tati/Code/cosmos-clean/astronomer-cosmos/venv-210/lib/python3.10/site-packages/google/api_core/future/polling.py", line 261, in result raise self._exception google.api_core.exceptions.NotFound: 404 Not found: Dataset astronomer-dag-authoring:release_17 was not found in location US; reason: notFound, message: Not found: Dataset astronomer-dag-authoring:release_17 was not found in location US Location: US Job ID: airflow_1727777155412089_041416781bf829a03b1f9b8fbc79980d --- cosmos/operators/airflow_async.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index 4089795de..1752abb54 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -60,7 +60,10 @@ def __init__(self, *args, full_refresh: bool = False, **kwargs): # type: ignore profile = self.profile_config.profile_mapping.profile self.gcp_project = profile["project"] self.dataset = profile["dataset"] - super().__init__(*args, configuration=self.configuration, task_id=kwargs["task_id"], deferrable=True) + self.location = kwargs.get("location", "northamerica-northeast1") # TODO: must be provided by users + super().__init__( + *args, configuration=self.configuration, location=self.location, task_id=kwargs["task_id"], deferrable=True + ) if self.profile_type not in _SUPPORTED_DATABASES: raise CosmosValueError(f"Async run are only supported: {_SUPPORTED_DATABASES}") From 1195955143a87aa897457987b214a6cedf60c987 Mon Sep 17 00:00:00 2001 From: pankajastro Date: Tue, 1 Oct 2024 19:03:14 +0530 Subject: [PATCH 18/49] Add limitation in docs --- docs/getting_started/execution-modes.rst | 30 ++++++++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/docs/getting_started/execution-modes.rst b/docs/getting_started/execution-modes.rst index 64a73060c..4b4275147 100644 --- a/docs/getting_started/execution-modes.rst +++ b/docs/getting_started/execution-modes.rst @@ -272,6 +272,9 @@ Note that currently, the ``airflow_async`` execution mode has the following limi 1. Only supports the ``dbt resource type`` models to be run asynchronously using Airflow deferrable operators. All other resources are executed synchronously using dbt commands as they are in the ``local`` execution mode. 2. Only supports BigQuery as the target database. If a profile target other than BigQuery is specified, Cosmos will error out saying that the target database is not supported with this execution mode. 3. Only works for ``full_refresh`` models. There is pending work to support other modes. +4. Only Support for the Bigquery profile type +5. Users need to provide ProfileMapping parameter in ProfileConfig + Example DAG: @@ -280,6 +283,33 @@ Example DAG: :start-after: [START airflow_async_execution_mode_example] :end-before: [END airflow_async_execution_mode_example] +**Known Issue:** + +The ```dag test``` command failed with the following error, likely because the trigger does not fully initialize during the ```dag test```, leading to an uninitialized task instance. +This causes the BigQuery trigger to attempt accessing parameters of the Task Instance that are not properly initialized. + +.. code:: bash + + [2024-10-01T18:19:09.726+0530] {base_events.py:1738} ERROR - unhandled exception during asyncio.run() shutdown + task: ()> exception=AttributeError("'NoneType' object has no attribute 'dag_id'")> + Traceback (most recent call last): + File "/Users/pankaj/Documents/astro_code/astronomer-cosmos/devenv/lib/python3.9/site-packages/airflow/providers/google/cloud/triggers/bigquery.py", line 138, in run + yield TriggerEvent( + asyncio.exceptions.CancelledError + + During handling of the above exception, another exception occurred: + + Traceback (most recent call last): + File "/Users/pankaj/Documents/astro_code/astronomer-cosmos/devenv/lib/python3.9/site-packages/airflow/providers/google/cloud/triggers/bigquery.py", line 157, in run + if self.job_id and self.cancel_on_kill and self.safe_to_cancel(): + File "/Users/pankaj/Documents/astro_code/astronomer-cosmos/devenv/lib/python3.9/site-packages/airflow/providers/google/cloud/triggers/bigquery.py", line 126, in safe_to_cancel + task_instance = self.get_task_instance() # type: ignore[call-arg] + File "/Users/pankaj/Documents/astro_code/astronomer-cosmos/devenv/lib/python3.9/site-packages/airflow/utils/session.py", line 97, in wrapper + return func(*args, session=session, **kwargs) + File "/Users/pankaj/Documents/astro_code/astronomer-cosmos/devenv/lib/python3.9/site-packages/airflow/providers/google/cloud/triggers/bigquery.py", line 102, in get_task_instance + TaskInstance.dag_id == self.task_instance.dag_id, + AttributeError: 'NoneType' object has no attribute 'dag_id' + .. _invocation_modes: Invocation Modes ================ From 2bdd9bb2b71715ce783ac7d8343d819343ec2845 Mon Sep 17 00:00:00 2001 From: pankajastro Date: Tue, 1 Oct 2024 19:20:22 +0530 Subject: [PATCH 19/49] Add full_refresh as templated field --- cosmos/operators/airflow_async.py | 5 ++++- docs/getting_started/execution-modes.rst | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index 1752abb54..ced0daf53 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -1,6 +1,6 @@ from __future__ import annotations -from typing import Any +from typing import Any, Sequence from airflow.providers.google.cloud.hooks.bigquery import BigQueryHook from airflow.providers.google.cloud.operators.bigquery import BigQueryInsertJobOperator @@ -44,6 +44,9 @@ class DbtSourceAirflowAsyncOperator(DbtSourceLocalOperator): class DbtRunAirflowAsyncOperator(BigQueryInsertJobOperator): # type: ignore + + template_fields: Sequence[str] = ("full_refresh",) + def __init__(self, *args, full_refresh: bool = False, **kwargs): # type: ignore # dbt task param self.profile_config = kwargs.get("profile_config") diff --git a/docs/getting_started/execution-modes.rst b/docs/getting_started/execution-modes.rst index 4b4275147..0c1b2f54d 100644 --- a/docs/getting_started/execution-modes.rst +++ b/docs/getting_started/execution-modes.rst @@ -285,7 +285,7 @@ Example DAG: **Known Issue:** -The ```dag test``` command failed with the following error, likely because the trigger does not fully initialize during the ```dag test```, leading to an uninitialized task instance. +The ``dag test`` command failed with the following error, likely because the trigger does not fully initialize during the ``dag test``, leading to an uninitialized task instance. This causes the BigQuery trigger to attempt accessing parameters of the Task Instance that are not properly initialized. .. code:: bash From 4a446035354257931df735d4cf6bedf7778db2da Mon Sep 17 00:00:00 2001 From: pankajastro Date: Tue, 1 Oct 2024 19:33:51 +0530 Subject: [PATCH 20/49] Add more template fields --- cosmos/operators/airflow_async.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index ced0daf53..680587d39 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -45,7 +45,13 @@ class DbtSourceAirflowAsyncOperator(DbtSourceLocalOperator): class DbtRunAirflowAsyncOperator(BigQueryInsertJobOperator): # type: ignore - template_fields: Sequence[str] = ("full_refresh",) + template_fields: Sequence[str] = ( + "full_refresh", + "project_dir", + "gcp_project", + "dataset", + "location", + ) def __init__(self, *args, full_refresh: bool = False, **kwargs): # type: ignore # dbt task param From c3c51cb1355183b5b1577b12da9195ebffed5f09 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Tue, 1 Oct 2024 23:51:27 +0530 Subject: [PATCH 21/49] Construct & relay 'dbt dag-task group' identifier to upload & download compile SQL --- cosmos/airflow/graph.py | 23 +++++++++++++++++++++-- cosmos/operators/airflow_async.py | 12 +++++++----- cosmos/operators/local.py | 21 +++++++-------------- 3 files changed, 35 insertions(+), 21 deletions(-) diff --git a/cosmos/airflow/graph.py b/cosmos/airflow/graph.py index c94fd0246..69d44ca98 100644 --- a/cosmos/airflow/graph.py +++ b/cosmos/airflow/graph.py @@ -134,6 +134,7 @@ def create_task_metadata( node: DbtNode, execution_mode: ExecutionMode, args: dict[str, Any], + dbt_dag_task_group_identifier: str, use_task_group: bool = False, source_rendering_behavior: SourceRenderingBehavior = SourceRenderingBehavior.NONE, ) -> TaskMetadata | None: @@ -144,6 +145,7 @@ def create_task_metadata( :param execution_mode: Where Cosmos should run each dbt task (e.g. ExecutionMode.LOCAL, ExecutionMode.KUBERNETES). Default is ExecutionMode.LOCAL. :param args: Arguments to be used to instantiate an Airflow Task + :param dbt_dag_task_group_identifier: Identifier to refer to the DbtDAG or DbtTaskGroup in the DAG. :param use_task_group: It determines whether to use the name as a prefix for the task id or not. If it is False, then use the name as a prefix for the task id, otherwise do not. :returns: The metadata necessary to instantiate the source dbt node as an Airflow task. @@ -158,7 +160,10 @@ def create_task_metadata( args = {**args, **{"models": node.resource_name}} if DbtResourceType(node.resource_type) in DEFAULT_DBT_RESOURCES and node.resource_type in dbt_resource_to_class: - extra_context = {"dbt_node_config": node.context_dict} + extra_context = { + "dbt_node_config": node.context_dict, + "dbt_dag_task_group_identifier": dbt_dag_task_group_identifier, + } if node.resource_type == DbtResourceType.MODEL: task_id = f"{node.name}_run" if use_task_group is True: @@ -228,6 +233,7 @@ def generate_task_or_group( node=node, execution_mode=execution_mode, args=task_args, + dbt_dag_task_group_identifier=_get_dbt_dag_task_group_identifier(dag, task_group), use_task_group=use_task_group, source_rendering_behavior=source_rendering_behavior, ) @@ -270,7 +276,7 @@ def _add_dbt_compile_task( id=DBT_COMPILE_TASK_ID, operator_class="cosmos.operators.airflow_async.DbtCompileAirflowAsyncOperator", arguments=task_args, - extra_context={}, + extra_context={"dbt_dag_task_group_identifier": _get_dbt_dag_task_group_identifier(dag, task_group)}, ) compile_airflow_task = create_airflow_task(compile_task_metadata, dag, task_group=task_group) tasks_map[DBT_COMPILE_TASK_ID] = compile_airflow_task @@ -280,6 +286,19 @@ def _add_dbt_compile_task( tasks_map[DBT_COMPILE_TASK_ID] >> tasks_map[node_id] +def _get_dbt_dag_task_group_identifier(dag: DAG, task_group: TaskGroup | None) -> str: + dag_id = dag.dag_id + task_group_id = task_group.group_id if task_group else None + identifiers_list = [] + if dag_id: + identifiers_list.append(dag_id) + if task_group_id: + identifiers_list.append(task_group_id) + dag_task_group_identifier = "__".join(identifiers_list) + + return dag_task_group_identifier + + def build_airflow_graph( nodes: dict[str, DbtNode], dag: DAG, # Airflow-specific - parent DAG where to associate tasks and (optional) task groups diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index 680587d39..bde91fa6a 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -57,7 +57,7 @@ def __init__(self, *args, full_refresh: bool = False, **kwargs): # type: ignore # dbt task param self.profile_config = kwargs.get("profile_config") self.project_dir = kwargs.get("project_dir") - self.file_path = kwargs.get("extra_context", {}).get("dbt_node_config", {}).get("file_path") + self.extra_context = kwargs.get("extra_context", {}) self.profile_type: str = self.profile_config.get_profile_type() # type: ignore self.full_refresh = full_refresh @@ -82,11 +82,13 @@ def get_remote_sql(self) -> str: raise CosmosValueError(f"Cosmos async support is only available starting in Airflow 2.8 or later.") from airflow.io.path import ObjectStoragePath - if not self.file_path or not self.project_dir: - raise CosmosValueError("file_path and project_dir are required to be set on the task for async execution") + file_path = self.extra_context["dbt_node_config"]["file_path"] + dbt_dag_task_group_identifier = self.extra_context["dbt_dag_task_group_identifier"] + + remote_target_path_str = str(remote_target_path).rstrip("/") project_dir_parent = str(self.project_dir.parent) - relative_file_path = str(self.file_path).replace(project_dir_parent, "").lstrip("/") - remote_model_path = f"{str(remote_target_path).rstrip('/')}/{self.dag_id}/compiled/{relative_file_path}" + relative_file_path = str(file_path).replace(project_dir_parent, "").lstrip("/") + remote_model_path = f"{remote_target_path_str}/{dbt_dag_task_group_identifier}/compiled/{relative_file_path}" print("remote_model_path: ", remote_model_path) object_storage_path = ObjectStoragePath(remote_model_path, conn_id=remote_target_path_conn_id) diff --git a/cosmos/operators/local.py b/cosmos/operators/local.py index b40cb411c..5a2d97c23 100644 --- a/cosmos/operators/local.py +++ b/cosmos/operators/local.py @@ -313,23 +313,16 @@ def _configure_remote_target_path() -> tuple[Path, str] | tuple[None, None]: return _configured_target_path, remote_conn_id def _construct_dest_file_path( - self, dest_target_dir: Path, file_path: str, source_compiled_dir: Path, context: Context + self, + dest_target_dir: Path, + file_path: str, + source_compiled_dir: Path, ) -> str: """ Construct the destination path for the compiled SQL files to be uploaded to the remote store. """ dest_target_dir_str = str(dest_target_dir).rstrip("/") - - task = context["task"] - dag_id = task.dag_id - task_group_id = task.task_group.group_id if task.task_group else None - identifiers_list = [] - if dag_id: - identifiers_list.append(dag_id) - if task_group_id: - identifiers_list.append(task_group_id) - dag_task_group_identifier = "__".join(identifiers_list) - + dag_task_group_identifier = self.extra_context["dbt_dag_task_group_identifier"] rel_path = os.path.relpath(file_path, source_compiled_dir).lstrip("/") return f"{dest_target_dir_str}/{dag_task_group_identifier}/compiled/{rel_path}" @@ -353,7 +346,7 @@ def upload_compiled_sql(self, tmp_project_dir: str, context: Context) -> None: source_compiled_dir = Path(tmp_project_dir) / "target" / "compiled" files = [str(file) for file in source_compiled_dir.rglob("*") if file.is_file()] for file_path in files: - dest_file_path = self._construct_dest_file_path(dest_target_dir, file_path, source_compiled_dir, context) + dest_file_path = self._construct_dest_file_path(dest_target_dir, file_path, source_compiled_dir) dest_object_storage_path = ObjectStoragePath(dest_file_path, conn_id=dest_conn_id) ObjectStoragePath(file_path).copy(dest_object_storage_path) self.log.debug("Copied %s to %s", file_path, dest_object_storage_path) @@ -1012,5 +1005,5 @@ def __init__(self, **kwargs: str) -> None: class DbtCompileLocalOperator(DbtCompileMixin, DbtLocalBaseOperator): def __init__(self, *args: Any, **kwargs: Any) -> None: - kwargs["should_upload_compiled_sql"] = False + kwargs["should_upload_compiled_sql"] = True super().__init__(*args, **kwargs) From 72c6164480c553b12c366bc11fc9d9e049515100 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 00:37:35 +0530 Subject: [PATCH 22/49] Fix model_name retrieval; get from dbt_node_config --- cosmos/operators/airflow_async.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index bde91fa6a..824b57468 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -96,7 +96,7 @@ def get_remote_sql(self) -> str: return fp.read() # type: ignore def drop_table_sql(self) -> None: - model_name = self.task_id.split(".")[0] + model_name = self.extra_context["dbt_node_config"]["resource_name"] sql = f"DROP TABLE IF EXISTS {self.gcp_project}.{self.dataset}.{model_name};" hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, @@ -117,7 +117,7 @@ def execute(self, context: Context) -> Any | None: self.drop_table_sql() sql = self.get_remote_sql() - model_name = self.task_id.split(".")[0] + model_name = self.extra_context["dbt_node_config"]["resource_name"] # prefix explicit create command to create table sql = f"CREATE TABLE {self.gcp_project}.{self.dataset}.{model_name} AS {sql}" From e67098e843d7017618bfd9940bfad207980520c8 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 01:38:07 +0530 Subject: [PATCH 23/49] Fix unit tests --- tests/airflow/test_graph.py | 33 +++++++++++++++++++++++++-------- tests/dbt/test_graph.py | 4 ++-- 2 files changed, 27 insertions(+), 10 deletions(-) diff --git a/tests/airflow/test_graph.py b/tests/airflow/test_graph.py index 6fc7cdc0a..d1266d81d 100644 --- a/tests/airflow/test_graph.py +++ b/tests/airflow/test_graph.py @@ -318,7 +318,7 @@ def test_create_task_metadata_unsupported(caplog): tags=[], config={}, ) - response = create_task_metadata(child_node, execution_mode="", args={}) + response = create_task_metadata(child_node, execution_mode="", args={}, dbt_dag_task_group_identifier="") assert response is None expected_msg = ( "Unavailable conversion function for (node ). " @@ -337,6 +337,7 @@ def test_create_task_metadata_unsupported(caplog): "cosmos.operators.local.DbtRunLocalOperator", {"models": "my_model"}, { + "dbt_dag_task_group_identifier": "", "dbt_node_config": { "unique_id": "model.my_folder.my_model", "resource_type": "model", @@ -347,7 +348,7 @@ def test_create_task_metadata_unsupported(caplog): "has_test": False, "resource_name": "my_model", "name": "my_model", - } + }, }, ), ( @@ -377,6 +378,7 @@ def test_create_task_metadata_unsupported(caplog): "cosmos.operators.local.DbtSnapshotLocalOperator", {"models": "my_snapshot"}, { + "dbt_dag_task_group_identifier": "", "dbt_node_config": { "unique_id": "snapshot.my_folder.my_snapshot", "resource_type": "snapshot", @@ -411,7 +413,9 @@ def test_create_task_metadata_model( has_freshness=True, ) - metadata = create_task_metadata(child_node, execution_mode=ExecutionMode.LOCAL, args={}) + metadata = create_task_metadata( + child_node, execution_mode=ExecutionMode.LOCAL, args={}, dbt_dag_task_group_identifier="" + ) if metadata: assert metadata.id == expected_id assert metadata.operator_class == expected_operator_class @@ -428,7 +432,9 @@ def test_create_task_metadata_model_with_versions(caplog): tags=[], config={}, ) - metadata = create_task_metadata(child_node, execution_mode=ExecutionMode.LOCAL, args={}) + metadata = create_task_metadata( + child_node, execution_mode=ExecutionMode.LOCAL, args={}, dbt_dag_task_group_identifier="" + ) assert metadata.id == "my_model_v1_run" assert metadata.operator_class == "cosmos.operators.local.DbtRunLocalOperator" assert metadata.arguments == {"models": "my_model.v1"} @@ -443,7 +449,9 @@ def test_create_task_metadata_model_use_task_group(caplog): tags=[], config={}, ) - metadata = create_task_metadata(child_node, execution_mode=ExecutionMode.LOCAL, args={}, use_task_group=True) + metadata = create_task_metadata( + child_node, execution_mode=ExecutionMode.LOCAL, args={}, use_task_group=True, dbt_dag_task_group_identifier="" + ) assert metadata.id == "run" @@ -498,7 +506,11 @@ def test_create_task_metadata_source_with_rendering_options( ) metadata = create_task_metadata( - child_node, execution_mode=ExecutionMode.LOCAL, source_rendering_behavior=source_rendering_behavior, args={} + child_node, + execution_mode=ExecutionMode.LOCAL, + source_rendering_behavior=source_rendering_behavior, + args={}, + dbt_dag_task_group_identifier="", ) if metadata: assert metadata.id == expected_id @@ -516,12 +528,15 @@ def test_create_task_metadata_seed(caplog, use_task_group): config={}, ) if use_task_group is None: - metadata = create_task_metadata(sample_node, execution_mode=ExecutionMode.DOCKER, args={}) + metadata = create_task_metadata( + sample_node, execution_mode=ExecutionMode.DOCKER, args={}, dbt_dag_task_group_identifier="" + ) else: metadata = create_task_metadata( sample_node, execution_mode=ExecutionMode.DOCKER, args={}, + dbt_dag_task_group_identifier="", use_task_group=use_task_group, ) @@ -543,7 +558,9 @@ def test_create_task_metadata_snapshot(caplog): tags=[], config={}, ) - metadata = create_task_metadata(sample_node, execution_mode=ExecutionMode.KUBERNETES, args={}) + metadata = create_task_metadata( + sample_node, execution_mode=ExecutionMode.KUBERNETES, args={}, dbt_dag_task_group_identifier="" + ) assert metadata.id == "my_snapshot_snapshot" assert metadata.operator_class == "cosmos.operators.kubernetes.DbtSnapshotKubernetesOperator" assert metadata.arguments == {"models": "my_snapshot"} diff --git a/tests/dbt/test_graph.py b/tests/dbt/test_graph.py index 4174c9a2d..30ab0e21a 100644 --- a/tests/dbt/test_graph.py +++ b/tests/dbt/test_graph.py @@ -1522,9 +1522,9 @@ def test_save_dbt_ls_cache(mock_variable_set, mock_datetime, tmp_dbt_project_dir hash_dir, hash_args = version.split(",") assert hash_args == "d41d8cd98f00b204e9800998ecf8427e" if sys.platform == "darwin": - assert hash_dir == "c1e25b0679b5ddcb636bcc30f2f85a06" + assert hash_dir == "25beeb54cc4eeabe6198248e286a1cfe" else: - assert hash_dir == "6f63493009733a7be34364a6ea3ffd3c" + assert hash_dir == "45ed7a4e1ab1ede62ccbe7ebdaa14208" @pytest.mark.integration From 3e550bfb211609bcc3526ff793ac7168d0712c2f Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 01:55:58 +0530 Subject: [PATCH 24/49] Fix subsequent failing unit tests --- tests/operators/test_airflow_async.py | 4 ++-- tests/operators/test_local.py | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/operators/test_airflow_async.py b/tests/operators/test_airflow_async.py index ef58a4d48..ec2f5e715 100644 --- a/tests/operators/test_airflow_async.py +++ b/tests/operators/test_airflow_async.py @@ -1,5 +1,6 @@ import pytest from airflow import __version__ as airflow_version +from airflow.providers.google.cloud.operators.bigquery import BigQueryInsertJobOperator from packaging import version from cosmos.operators.airflow_async import ( @@ -17,7 +18,6 @@ DbtBuildLocalOperator, DbtCompileLocalOperator, DbtLSLocalOperator, - DbtRunLocalOperator, DbtRunOperationLocalOperator, DbtSeedLocalOperator, DbtSnapshotLocalOperator, @@ -51,7 +51,7 @@ def test_dbt_source_airflow_async_operator_inheritance(): reason="Cosmos Async operators only work with Airflow 2.8 onwards.", ) def test_dbt_run_airflow_async_operator_inheritance(): - assert issubclass(DbtRunAirflowAsyncOperator, DbtRunLocalOperator) + assert issubclass(DbtRunAirflowAsyncOperator, BigQueryInsertJobOperator) def test_dbt_test_airflow_async_operator_inheritance(): diff --git a/tests/operators/test_local.py b/tests/operators/test_local.py index bfd1408d9..ed954dfdf 100644 --- a/tests/operators/test_local.py +++ b/tests/operators/test_local.py @@ -1242,6 +1242,7 @@ def test_upload_compiled_sql_should_upload(mock_configure_remote, mock_object_st profile_config=profile_config, project_dir="fake-dir", dag=DAG("test_dag", start_date=datetime(2024, 4, 16)), + extra_context={"dbt_dag_task_group_identifier": "test_dag"}, ) mock_configure_remote.return_value = ("mock_remote_path", "mock_conn_id") From 0730d0fb0b8051ec8caca38ad5e0cb5c9e04f8ae Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 02:01:30 +0530 Subject: [PATCH 25/49] Fix type check failures --- cosmos/airflow/graph.py | 4 ++-- cosmos/core/airflow.py | 3 ++- cosmos/core/graph/entities.py | 2 +- cosmos/operators/airflow_async.py | 6 +++++- 4 files changed, 10 insertions(+), 5 deletions(-) diff --git a/cosmos/airflow/graph.py b/cosmos/airflow/graph.py index 69d44ca98..595e080c4 100644 --- a/cosmos/airflow/graph.py +++ b/cosmos/airflow/graph.py @@ -81,7 +81,7 @@ def create_test_task_metadata( on_warning_callback: Callable[..., Any] | None = None, node: DbtNode | None = None, render_config: RenderConfig | None = None, - async_op_args: dict | None = None, + async_op_args: dict[str, Any] | None = None, ) -> TaskMetadata: """ Create the metadata that will be used to instantiate the Airflow Task that will be used to run the Dbt test node. @@ -309,7 +309,7 @@ def build_airflow_graph( render_config: RenderConfig, task_group: TaskGroup | None = None, on_warning_callback: Callable[..., Any] | None = None, # argument specific to the DBT test command - async_op_args: dict | None = None, + async_op_args: dict[str, Any] | None = None, ) -> None: """ Instantiate dbt `nodes` as Airflow tasks within the given `task_group` (optional) or `dag` (mandatory). diff --git a/cosmos/core/airflow.py b/cosmos/core/airflow.py index 67230fb39..00b1c252f 100644 --- a/cosmos/core/airflow.py +++ b/cosmos/core/airflow.py @@ -1,6 +1,7 @@ from __future__ import annotations import importlib +from typing import Any from airflow.models import BaseOperator from airflow.models.dag import DAG @@ -27,7 +28,7 @@ def get_airflow_task(task: Task, dag: DAG, task_group: TaskGroup | None = None) module = importlib.import_module(module_name) Operator = getattr(module, class_name) - task_kwargs = {} + task_kwargs: dict[str, Any] = {} if task.owner != "": task_kwargs["owner"] = task.owner diff --git a/cosmos/core/graph/entities.py b/cosmos/core/graph/entities.py index 340cb7781..c0099ad89 100644 --- a/cosmos/core/graph/entities.py +++ b/cosmos/core/graph/entities.py @@ -61,4 +61,4 @@ class Task(CosmosEntity): operator_class: str = "airflow.operators.empty.EmptyOperator" arguments: Dict[str, Any] = field(default_factory=dict) extra_context: Dict[str, Any] = field(default_factory=dict) - async_op_args: Dict[str, Any] = field(default_factory=dict) + async_op_args: Dict[str, Any] | None = field(default_factory=dict) diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index 824b57468..17c84a798 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -1,6 +1,6 @@ from __future__ import annotations -from typing import Any, Sequence +from typing import TYPE_CHECKING, Any, Sequence from airflow.providers.google.cloud.hooks.bigquery import BigQueryHook from airflow.providers.google.cloud.operators.bigquery import BigQueryInsertJobOperator @@ -86,6 +86,10 @@ def get_remote_sql(self) -> str: dbt_dag_task_group_identifier = self.extra_context["dbt_dag_task_group_identifier"] remote_target_path_str = str(remote_target_path).rstrip("/") + + if TYPE_CHECKING: + assert self.project_dir is not None + project_dir_parent = str(self.project_dir.parent) relative_file_path = str(file_path).replace(project_dir_parent, "").lstrip("/") remote_model_path = f"{remote_target_path_str}/{dbt_dag_task_group_identifier}/compiled/{relative_file_path}" From 745768e9f87109fb292bb6a798b46b465fcdf6d3 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 02:22:06 +0530 Subject: [PATCH 26/49] Add back the deleted sources.yml from jaffle_shop as it has dependencies in models --- .../jaffle_shop/models/staging/sources.yml | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) create mode 100644 dev/dags/dbt/jaffle_shop/models/staging/sources.yml diff --git a/dev/dags/dbt/jaffle_shop/models/staging/sources.yml b/dev/dags/dbt/jaffle_shop/models/staging/sources.yml new file mode 100644 index 000000000..a3139b585 --- /dev/null +++ b/dev/dags/dbt/jaffle_shop/models/staging/sources.yml @@ -0,0 +1,31 @@ + +version: 2 + +sources: + - name: postgres_db + database: "{{ env_var('POSTGRES_DB') }}" + schema: "{{ env_var('POSTGRES_SCHEMA') }}" + tables: + - name: raw_customers + columns: + - name: id + tests: + - unique + - not_null + - name: raw_payments + columns: + - name: id + tests: + - unique + - not_null + - name: raw_orders + columns: + - name: id + tests: + - unique + - not_null + freshness: + warn_after: + count: 3650 + period: day + loaded_at_field: CAST(order_date AS TIMESTAMP) From 43d62eaf49532b913b31a88a73fc4ed52fd1ee57 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 02:37:41 +0530 Subject: [PATCH 27/49] Install dbt bigquery adapter for running simple_dag_async --- dev/Dockerfile | 2 +- scripts/test/integration-setup.sh | 2 +- tests/dbt/test_graph.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/Dockerfile b/dev/Dockerfile index a17bb9943..9fd3df75a 100644 --- a/dev/Dockerfile +++ b/dev/Dockerfile @@ -17,7 +17,7 @@ COPY ./README.rst ${AIRFLOW_HOME}/astronomer_cosmos/ COPY ./cosmos/ ${AIRFLOW_HOME}/astronomer_cosmos/cosmos/ COPY ./dev/requirements.txt ${AIRFLOW_HOME}/requirements.txt # install the package in editable mode -RUN uv pip install --system -e "${AIRFLOW_HOME}/astronomer_cosmos"[dbt-postgres,dbt-databricks] && \ +RUN uv pip install --system -e "${AIRFLOW_HOME}/astronomer_cosmos"[dbt-postgres,dbt-databricks,dbt-bigquery] && \ uv pip install --system -r ${AIRFLOW_HOME}/requirements.txt diff --git a/scripts/test/integration-setup.sh b/scripts/test/integration-setup.sh index c6e106fd5..fec9e95eb 100644 --- a/scripts/test/integration-setup.sh +++ b/scripts/test/integration-setup.sh @@ -11,4 +11,4 @@ rm -rf airflow.* pip freeze | grep airflow airflow db reset -y airflow db init -pip install 'dbt-core' 'dbt-databricks' 'dbt-postgres' 'dbt-vertica' 'openlineage-airflow' +pip install 'dbt-core' 'dbt-bigquery' 'dbt-databricks' 'dbt-postgres' 'dbt-vertica' 'openlineage-airflow' diff --git a/tests/dbt/test_graph.py b/tests/dbt/test_graph.py index 30ab0e21a..1c0912042 100644 --- a/tests/dbt/test_graph.py +++ b/tests/dbt/test_graph.py @@ -1524,7 +1524,7 @@ def test_save_dbt_ls_cache(mock_variable_set, mock_datetime, tmp_dbt_project_dir if sys.platform == "darwin": assert hash_dir == "25beeb54cc4eeabe6198248e286a1cfe" else: - assert hash_dir == "45ed7a4e1ab1ede62ccbe7ebdaa14208" + assert hash_dir == "6f63493009733a7be34364a6ea3ffd3c" @pytest.mark.integration From 9656248a3e720cc245b56852fa953438d3ff85cd Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 03:20:44 +0530 Subject: [PATCH 28/49] Install dbt bigquery adapter in our CI setup scripts --- scripts/test/integration-dbt-1-5-4.sh | 2 +- scripts/test/integration-sqlite-setup.sh | 2 +- tests/airflow/test_graph.py | 21 +++++++++++---------- 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/scripts/test/integration-dbt-1-5-4.sh b/scripts/test/integration-dbt-1-5-4.sh index bb936fc21..6992b8f15 100644 --- a/scripts/test/integration-dbt-1-5-4.sh +++ b/scripts/test/integration-dbt-1-5-4.sh @@ -1,5 +1,5 @@ pip uninstall dbt-adapters dbt-common dbt-core dbt-extractor dbt-postgres dbt-semantic-interfaces -y -pip install dbt-postgres==1.5.4 dbt-databricks==1.5.4 +pip install dbt-postgres==1.5.4 dbt-databricks==1.5.4 dbt-bigquery==1.5.4 export SOURCE_RENDERING_BEHAVIOR=all rm -rf airflow.*; \ airflow db init; \ diff --git a/scripts/test/integration-sqlite-setup.sh b/scripts/test/integration-sqlite-setup.sh index b8bce035c..c7f547ce9 100644 --- a/scripts/test/integration-sqlite-setup.sh +++ b/scripts/test/integration-sqlite-setup.sh @@ -1,4 +1,4 @@ pip uninstall -y dbt-core dbt-sqlite openlineage-airflow openlineage-integration-common; \ rm -rf airflow.*; \ airflow db init; \ -pip install 'dbt-core==1.4' 'dbt-sqlite<=1.4' 'dbt-databricks<=1.4' 'dbt-postgres<=1.4' +pip install 'dbt-core==1.4' 'dbt-sqlite<=1.4' 'dbt-databricks<=1.4' 'dbt-postgres<=1.4' 'dbt-bigquery<=1.4' diff --git a/tests/airflow/test_graph.py b/tests/airflow/test_graph.py index d1266d81d..1b699cb02 100644 --- a/tests/airflow/test_graph.py +++ b/tests/airflow/test_graph.py @@ -1,7 +1,7 @@ import os from datetime import datetime from pathlib import Path -from unittest.mock import patch +from unittest.mock import MagicMock, patch import pytest from airflow import __version__ as airflow_version @@ -30,7 +30,7 @@ ) from cosmos.converter import airflow_kwargs from cosmos.dbt.graph import DbtNode -from cosmos.profiles import PostgresUserPasswordProfileMapping +from cosmos.profiles import GoogleCloudServiceAccountFileProfileMapping, PostgresUserPasswordProfileMapping SAMPLE_PROJ_PATH = Path("/home/user/path/dbt-proj/") SOURCE_RENDERING_BEHAVIOR = SourceRenderingBehavior(os.getenv("SOURCE_RENDERING_BEHAVIOR", "none")) @@ -228,19 +228,20 @@ def test_build_airflow_graph_with_after_all(): @pytest.mark.integration +@patch("airflow.hooks.base.BaseHook.get_connection", new=MagicMock()) def test_build_airflow_graph_with_dbt_compile_task(): + bigquery_profile_config = ProfileConfig( + profile_name="my-bigquery-db", + target_name="dev", + profile_mapping=GoogleCloudServiceAccountFileProfileMapping( + conn_id="fake_conn", profile_args={"dataset": "release_17"} + ), + ) with DAG("test-id-dbt-compile", start_date=datetime(2022, 1, 1)) as dag: task_args = { "project_dir": SAMPLE_PROJ_PATH, "conn_id": "fake_conn", - "profile_config": ProfileConfig( - profile_name="default", - target_name="default", - profile_mapping=PostgresUserPasswordProfileMapping( - conn_id="fake_conn", - profile_args={"schema": "public"}, - ), - ), + "profile_config": bigquery_profile_config, } render_config = RenderConfig( select=["tag:some"], From a654f497de7cf60e1a857fbeae26a671951e7381 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 03:40:34 +0530 Subject: [PATCH 29/49] Update gcp conn in dev/dags/simple_dag_async.py --- dev/dags/simple_dag_async.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/dags/simple_dag_async.py b/dev/dags/simple_dag_async.py index e80145215..b70a6eedd 100644 --- a/dev/dags/simple_dag_async.py +++ b/dev/dags/simple_dag_async.py @@ -12,7 +12,7 @@ profile_name="default", target_name="dev", profile_mapping=GoogleCloudServiceAccountFileProfileMapping( - conn_id="google_cloud_default", profile_args={"dataset": "release_17"} + conn_id="gcp_gs_conn", profile_args={"dataset": "release_17"} ), ) From e60ace2a522001ae7c14c55a30cb27173105cf84 Mon Sep 17 00:00:00 2001 From: Tatiana Al-Chueyr Date: Wed, 2 Oct 2024 00:09:04 +0100 Subject: [PATCH 30/49] Refactor args in DbtRunAirflowAsyncOperator --- cosmos/operators/airflow_async.py | 94 ++++++++++++++++++++++--------- cosmos/operators/local.py | 2 +- dev/dags/simple_dag_async.py | 2 +- 3 files changed, 68 insertions(+), 30 deletions(-) diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index 17c84a798..aa6733652 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -1,5 +1,7 @@ from __future__ import annotations +import inspect +from pathlib import Path from typing import TYPE_CHECKING, Any, Sequence from airflow.providers.google.cloud.hooks.bigquery import BigQueryHook @@ -7,10 +9,13 @@ from airflow.utils.context import Context from cosmos import settings +from cosmos.config import ProfileConfig from cosmos.exceptions import CosmosValueError +from cosmos.operators.base import AbstractDbtBaseOperator from cosmos.operators.local import ( DbtBuildLocalOperator, DbtCompileLocalOperator, + DbtLocalBaseOperator, DbtLSLocalOperator, DbtRunOperationLocalOperator, DbtSeedLocalOperator, @@ -22,24 +27,35 @@ _SUPPORTED_DATABASES = ["bigquery"] +from abc import ABCMeta -class DbtBuildAirflowAsyncOperator(DbtBuildLocalOperator): +from airflow.models.baseoperator import BaseOperator + + +class DbtBaseAirflowAsyncOperator(BaseOperator, metaclass=ABCMeta): + def __init__(self, **kwargs) -> None: # type: ignore + self.location = kwargs.pop("location") + self.configuration = kwargs.pop("configuration", {}) + super().__init__(**kwargs) + + +class DbtBuildAirflowAsyncOperator(DbtBaseAirflowAsyncOperator, DbtBuildLocalOperator): # type: ignore pass -class DbtLSAirflowAsyncOperator(DbtLSLocalOperator): +class DbtLSAirflowAsyncOperator(DbtBaseAirflowAsyncOperator, DbtLSLocalOperator): # type: ignore pass -class DbtSeedAirflowAsyncOperator(DbtSeedLocalOperator): +class DbtSeedAirflowAsyncOperator(DbtBaseAirflowAsyncOperator, DbtSeedLocalOperator): # type: ignore pass -class DbtSnapshotAirflowAsyncOperator(DbtSnapshotLocalOperator): +class DbtSnapshotAirflowAsyncOperator(DbtBaseAirflowAsyncOperator, DbtSnapshotLocalOperator): # type: ignore pass -class DbtSourceAirflowAsyncOperator(DbtSourceLocalOperator): +class DbtSourceAirflowAsyncOperator(DbtBaseAirflowAsyncOperator, DbtSourceLocalOperator): # type: ignore pass @@ -53,36 +69,56 @@ class DbtRunAirflowAsyncOperator(BigQueryInsertJobOperator): # type: ignore "location", ) - def __init__(self, *args, full_refresh: bool = False, **kwargs): # type: ignore + def __init__( # type: ignore + self, + project_dir: str, + profile_config: ProfileConfig, + location: str, # This is a mandatory parameter when using BigQueryInsertJobOperator with deferrable=True + full_refresh: bool = False, + extra_context: dict[str, object] | None = None, + configuration: dict[str, object] | None = None, + **kwargs, + ) -> None: # dbt task param - self.profile_config = kwargs.get("profile_config") - self.project_dir = kwargs.get("project_dir") - self.extra_context = kwargs.get("extra_context", {}) - self.profile_type: str = self.profile_config.get_profile_type() # type: ignore + self.project_dir = project_dir + self.extra_context = extra_context or {} self.full_refresh = full_refresh + self.profile_config = profile_config + if not self.profile_config or not self.profile_config.profile_mapping: + raise CosmosValueError(f"Cosmos async support is only available when using ProfileMapping") + + self.profile_type: str = profile_config.get_profile_type() # type: ignore + if self.profile_type not in _SUPPORTED_DATABASES: + raise CosmosValueError(f"Async run are only supported: {_SUPPORTED_DATABASES}") # airflow task param - self.configuration: dict[str, object] = {} + self.location = location + self.configuration = configuration or {} self.gcp_conn_id = self.profile_config.profile_mapping.conn_id # type: ignore - if not self.profile_config or not self.profile_config.profile_mapping: - raise CosmosValueError(f"Cosmos async support is only available starting in Airflow 2.8 or later.") profile = self.profile_config.profile_mapping.profile self.gcp_project = profile["project"] self.dataset = profile["dataset"] - self.location = kwargs.get("location", "northamerica-northeast1") # TODO: must be provided by users - super().__init__( - *args, configuration=self.configuration, location=self.location, task_id=kwargs["task_id"], deferrable=True - ) - if self.profile_type not in _SUPPORTED_DATABASES: - raise CosmosValueError(f"Async run are only supported: {_SUPPORTED_DATABASES}") + # Cosmos attempts to pass many kwargs that BigQueryInsertJobOperator simply does not accept. + # We need to pop them. + clean_kwargs = {} + non_async_args = set(inspect.signature(AbstractDbtBaseOperator.__init__).parameters.keys()) + non_async_args |= set(inspect.signature(DbtLocalBaseOperator.__init__).parameters.keys()) + non_async_args -= {"task_id"} + + for arg_key, arg_value in kwargs.items(): + if arg_key not in non_async_args: + clean_kwargs[arg_key] = arg_value + + # The following are the minimum required parameters to run BigQueryInsertJobOperator using the deferrable mode + super().__init__(configuration=self.configuration, location=self.location, deferrable=True, **clean_kwargs) def get_remote_sql(self) -> str: if not settings.AIRFLOW_IO_AVAILABLE: raise CosmosValueError(f"Cosmos async support is only available starting in Airflow 2.8 or later.") from airflow.io.path import ObjectStoragePath - file_path = self.extra_context["dbt_node_config"]["file_path"] + file_path = self.extra_context["dbt_node_config"]["file_path"] # type: ignore dbt_dag_task_group_identifier = self.extra_context["dbt_dag_task_group_identifier"] remote_target_path_str = str(remote_target_path).rstrip("/") @@ -90,7 +126,7 @@ def get_remote_sql(self) -> str: if TYPE_CHECKING: assert self.project_dir is not None - project_dir_parent = str(self.project_dir.parent) + project_dir_parent = str(Path(self.project_dir).parent) relative_file_path = str(file_path).replace(project_dir_parent, "").lstrip("/") remote_model_path = f"{remote_target_path_str}/{dbt_dag_task_group_identifier}/compiled/{relative_file_path}" @@ -100,7 +136,7 @@ def get_remote_sql(self) -> str: return fp.read() # type: ignore def drop_table_sql(self) -> None: - model_name = self.extra_context["dbt_node_config"]["resource_name"] + model_name = self.extra_context["dbt_node_config"]["resource_name"] # type: ignore sql = f"DROP TABLE IF EXISTS {self.gcp_project}.{self.dataset}.{model_name};" hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, @@ -118,13 +154,15 @@ def execute(self, context: Context) -> Any | None: if not self.full_refresh: raise CosmosValueError("The async execution only supported for full_refresh") else: + # It may be surprising to some, but the dbt-core --full-refresh argument fully drops the table before populating it + # https://github.com/dbt-labs/dbt-core/blob/5e9f1b515f37dfe6cdae1ab1aa7d190b92490e24/core/dbt/context/base.py#L662-L666 + # https://docs.getdbt.com/reference/resource-configs/full_refresh#recommendation + # We're emulating this behaviour here self.drop_table_sql() - sql = self.get_remote_sql() - model_name = self.extra_context["dbt_node_config"]["resource_name"] + model_name = self.extra_context["dbt_node_config"]["resource_name"] # type: ignore # prefix explicit create command to create table sql = f"CREATE TABLE {self.gcp_project}.{self.dataset}.{model_name} AS {sql}" - self.configuration = { "query": { "query": sql, @@ -134,13 +172,13 @@ def execute(self, context: Context) -> Any | None: return super().execute(context) -class DbtTestAirflowAsyncOperator(DbtTestLocalOperator): +class DbtTestAirflowAsyncOperator(DbtBaseAirflowAsyncOperator, DbtTestLocalOperator): # type: ignore pass -class DbtRunOperationAirflowAsyncOperator(DbtRunOperationLocalOperator): +class DbtRunOperationAirflowAsyncOperator(DbtBaseAirflowAsyncOperator, DbtRunOperationLocalOperator): # type: ignore pass -class DbtCompileAirflowAsyncOperator(DbtCompileLocalOperator): +class DbtCompileAirflowAsyncOperator(DbtBaseAirflowAsyncOperator, DbtCompileLocalOperator): # type: ignore pass diff --git a/cosmos/operators/local.py b/cosmos/operators/local.py index 5a2d97c23..11275062b 100644 --- a/cosmos/operators/local.py +++ b/cosmos/operators/local.py @@ -1005,5 +1005,5 @@ def __init__(self, **kwargs: str) -> None: class DbtCompileLocalOperator(DbtCompileMixin, DbtLocalBaseOperator): def __init__(self, *args: Any, **kwargs: Any) -> None: - kwargs["should_upload_compiled_sql"] = True + kwargs["should_upload_compiled_sql"] = False super().__init__(*args, **kwargs) diff --git a/dev/dags/simple_dag_async.py b/dev/dags/simple_dag_async.py index b70a6eedd..4af39cc01 100644 --- a/dev/dags/simple_dag_async.py +++ b/dev/dags/simple_dag_async.py @@ -37,6 +37,6 @@ catchup=False, dag_id="simple_dag_async", tags=["simple"], - operator_args={"install_deps": True, "full_refresh": True}, + operator_args={"install_deps": True, "full_refresh": True, "location": "northamerica-northeast1"}, ) # [END airflow_async_execution_mode_example] From 7f055bc7d6bb1363bc7cab5970cae2439696c10a Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 04:42:59 +0530 Subject: [PATCH 31/49] Use GoogleCloudServiceAccountDictProfileMapping in profilemapping --- dev/dags/simple_dag_async.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/dags/simple_dag_async.py b/dev/dags/simple_dag_async.py index 4af39cc01..1242682f3 100644 --- a/dev/dags/simple_dag_async.py +++ b/dev/dags/simple_dag_async.py @@ -3,7 +3,7 @@ from pathlib import Path from cosmos import DbtDag, ExecutionConfig, ExecutionMode, ProfileConfig, ProjectConfig, RenderConfig -from cosmos.profiles import GoogleCloudServiceAccountFileProfileMapping +from cosmos.profiles import GoogleCloudServiceAccountDictProfileMapping DEFAULT_DBT_ROOT_PATH = Path(__file__).parent / "dbt" DBT_ROOT_PATH = Path(os.getenv("DBT_ROOT_PATH", DEFAULT_DBT_ROOT_PATH)) @@ -11,7 +11,7 @@ profile_config = ProfileConfig( profile_name="default", target_name="dev", - profile_mapping=GoogleCloudServiceAccountFileProfileMapping( + profile_mapping=GoogleCloudServiceAccountDictProfileMapping( conn_id="gcp_gs_conn", profile_args={"dataset": "release_17"} ), ) From ad057c8943afe0d0c86ea343fb677ffc087daaaf Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 04:47:33 +0530 Subject: [PATCH 32/49] set should_upload_compiled_sql to True --- cosmos/operators/local.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cosmos/operators/local.py b/cosmos/operators/local.py index 11275062b..5a2d97c23 100644 --- a/cosmos/operators/local.py +++ b/cosmos/operators/local.py @@ -1005,5 +1005,5 @@ def __init__(self, **kwargs: str) -> None: class DbtCompileLocalOperator(DbtCompileMixin, DbtLocalBaseOperator): def __init__(self, *args: Any, **kwargs: Any) -> None: - kwargs["should_upload_compiled_sql"] = False + kwargs["should_upload_compiled_sql"] = True super().__init__(*args, **kwargs) From a70ca46bab506d68b0b1b204fa1d157d7b8d6ed0 Mon Sep 17 00:00:00 2001 From: Tatiana Al-Chueyr Date: Wed, 2 Oct 2024 00:22:44 +0100 Subject: [PATCH 33/49] Remove async_op_args --- cosmos/airflow/graph.py | 4 ---- cosmos/converter.py | 3 --- cosmos/core/airflow.py | 3 --- cosmos/core/graph/entities.py | 1 - cosmos/dbt/graph.py | 2 -- cosmos/operators/local.py | 2 -- 6 files changed, 15 deletions(-) diff --git a/cosmos/airflow/graph.py b/cosmos/airflow/graph.py index 595e080c4..fafa2e467 100644 --- a/cosmos/airflow/graph.py +++ b/cosmos/airflow/graph.py @@ -81,7 +81,6 @@ def create_test_task_metadata( on_warning_callback: Callable[..., Any] | None = None, node: DbtNode | None = None, render_config: RenderConfig | None = None, - async_op_args: dict[str, Any] | None = None, ) -> TaskMetadata: """ Create the metadata that will be used to instantiate the Airflow Task that will be used to run the Dbt test node. @@ -126,7 +125,6 @@ def create_test_task_metadata( ), arguments=task_args, extra_context=extra_context, - async_op_args=async_op_args, ) @@ -309,7 +307,6 @@ def build_airflow_graph( render_config: RenderConfig, task_group: TaskGroup | None = None, on_warning_callback: Callable[..., Any] | None = None, # argument specific to the DBT test command - async_op_args: dict[str, Any] | None = None, ) -> None: """ Instantiate dbt `nodes` as Airflow tasks within the given `task_group` (optional) or `dag` (mandatory). @@ -374,7 +371,6 @@ def build_airflow_graph( task_args=task_args, on_warning_callback=on_warning_callback, render_config=render_config, - async_op_args=async_op_args, ) test_task = create_airflow_task(test_meta, dag, task_group=task_group) leaves_ids = calculate_leaves(tasks_ids=list(tasks_map.keys()), nodes=nodes) diff --git a/cosmos/converter.py b/cosmos/converter.py index e4e0b7f6b..fd077c465 100644 --- a/cosmos/converter.py +++ b/cosmos/converter.py @@ -207,7 +207,6 @@ def __init__( task_group: TaskGroup | None = None, operator_args: dict[str, Any] | None = None, on_warning_callback: Callable[..., Any] | None = None, - async_op_args: dict[str, Any] | None = None, *args: Any, **kwargs: Any, ) -> None: @@ -257,7 +256,6 @@ def __init__( cache_identifier=cache_identifier, dbt_vars=dbt_vars, airflow_metadata=cache._get_airflow_metadata(dag, task_group), - async_op_args=async_op_args, ) self.dbt_graph.load(method=render_config.load_method, execution_mode=execution_config.execution_mode) @@ -303,7 +301,6 @@ def __init__( dbt_project_name=project_config.project_name, on_warning_callback=on_warning_callback, render_config=render_config, - async_op_args=async_op_args, ) current_time = time.perf_counter() diff --git a/cosmos/core/airflow.py b/cosmos/core/airflow.py index 00b1c252f..6f1064649 100644 --- a/cosmos/core/airflow.py +++ b/cosmos/core/airflow.py @@ -32,9 +32,6 @@ def get_airflow_task(task: Task, dag: DAG, task_group: TaskGroup | None = None) if task.owner != "": task_kwargs["owner"] = task.owner - if task.async_op_args: - task_kwargs["async_op_args"] = task.async_op_args - airflow_task = Operator( task_id=task.id, dag=dag, diff --git a/cosmos/core/graph/entities.py b/cosmos/core/graph/entities.py index c0099ad89..6bf9ff046 100644 --- a/cosmos/core/graph/entities.py +++ b/cosmos/core/graph/entities.py @@ -61,4 +61,3 @@ class Task(CosmosEntity): operator_class: str = "airflow.operators.empty.EmptyOperator" arguments: Dict[str, Any] = field(default_factory=dict) extra_context: Dict[str, Any] = field(default_factory=dict) - async_op_args: Dict[str, Any] | None = field(default_factory=dict) diff --git a/cosmos/dbt/graph.py b/cosmos/dbt/graph.py index 0400edc1a..7a957b2fc 100644 --- a/cosmos/dbt/graph.py +++ b/cosmos/dbt/graph.py @@ -217,7 +217,6 @@ def __init__( dbt_vars: dict[str, str] | None = None, airflow_metadata: dict[str, str] | None = None, operator_args: dict[str, Any] | None = None, - async_op_args: dict[str, Any] | None = None, ): self.project = project self.render_config = render_config @@ -225,7 +224,6 @@ def __init__( self.execution_config = execution_config self.cache_dir = cache_dir self.airflow_metadata = airflow_metadata or {} - self.async_op_args = async_op_args if cache_identifier: self.dbt_ls_cache_key = cache.create_cache_key(cache_identifier) else: diff --git a/cosmos/operators/local.py b/cosmos/operators/local.py index 5a2d97c23..05fa356f6 100644 --- a/cosmos/operators/local.py +++ b/cosmos/operators/local.py @@ -143,7 +143,6 @@ def __init__( should_store_compiled_sql: bool = True, should_upload_compiled_sql: bool = False, append_env: bool = True, - async_op_args: dict[str, Any] | None = None, **kwargs: Any, ) -> None: self.task_id = task_id @@ -158,7 +157,6 @@ def __init__( self.invoke_dbt: Callable[..., FullOutputSubprocessResult | dbtRunnerResult] self.handle_exception: Callable[..., None] self._dbt_runner: dbtRunner | None = None - self.async_op_args = async_op_args if self.invocation_mode: self._set_invocation_methods() From 7c6a1b2d5d335e17a4714170687d2b28f3e64755 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 04:54:30 +0530 Subject: [PATCH 34/49] remove install_deps from DAG --- dev/dags/simple_dag_async.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/dags/simple_dag_async.py b/dev/dags/simple_dag_async.py index 1242682f3..e9197e27b 100644 --- a/dev/dags/simple_dag_async.py +++ b/dev/dags/simple_dag_async.py @@ -37,6 +37,6 @@ catchup=False, dag_id="simple_dag_async", tags=["simple"], - operator_args={"install_deps": True, "full_refresh": True, "location": "northamerica-northeast1"}, + operator_args={"full_refresh": True, "location": "northamerica-northeast1"}, ) # [END airflow_async_execution_mode_example] From c1aeff05fc8d15d853e52cf597bc4bb5ea64bb0b Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 05:08:28 +0530 Subject: [PATCH 35/49] Fix test_build_airflow_graph_with_dbt_compile_task by passing needed location key --- tests/airflow/test_graph.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/airflow/test_graph.py b/tests/airflow/test_graph.py index 1b699cb02..1bd8cab35 100644 --- a/tests/airflow/test_graph.py +++ b/tests/airflow/test_graph.py @@ -242,6 +242,7 @@ def test_build_airflow_graph_with_dbt_compile_task(): "project_dir": SAMPLE_PROJ_PATH, "conn_id": "fake_conn", "profile_config": bigquery_profile_config, + "location": "", } render_config = RenderConfig( select=["tag:some"], From 02f798552f8d2b24384cf00d31e37c466b08c596 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 05:31:03 +0530 Subject: [PATCH 36/49] Specify required project id in the GoogleCloudServiceAccountDictProfileMapping in the DAG --- dev/dags/simple_dag_async.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/dags/simple_dag_async.py b/dev/dags/simple_dag_async.py index e9197e27b..1b2b67651 100644 --- a/dev/dags/simple_dag_async.py +++ b/dev/dags/simple_dag_async.py @@ -12,7 +12,7 @@ profile_name="default", target_name="dev", profile_mapping=GoogleCloudServiceAccountDictProfileMapping( - conn_id="gcp_gs_conn", profile_args={"dataset": "release_17"} + conn_id="gcp_gs_conn", profile_args={"dataset": "release_17", "project": "astronomer-dag-authoring"} ), ) From af454a924092b2472b340680ffc6885a5695a1ef Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 06:28:12 +0530 Subject: [PATCH 37/49] Pass gcp_conn_id to super class init, otherwise it is lost & uses the default google_cloud_default otherwise --- cosmos/operators/airflow_async.py | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index aa6733652..cb8838f97 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -111,7 +111,13 @@ def __init__( # type: ignore clean_kwargs[arg_key] = arg_value # The following are the minimum required parameters to run BigQueryInsertJobOperator using the deferrable mode - super().__init__(configuration=self.configuration, location=self.location, deferrable=True, **clean_kwargs) + super().__init__( + gcp_conn_id=self.gcp_conn_id, + configuration=self.configuration, + location=self.location, + deferrable=True, + **clean_kwargs, + ) def get_remote_sql(self) -> str: if not settings.AIRFLOW_IO_AVAILABLE: @@ -138,6 +144,13 @@ def get_remote_sql(self) -> str: def drop_table_sql(self) -> None: model_name = self.extra_context["dbt_node_config"]["resource_name"] # type: ignore sql = f"DROP TABLE IF EXISTS {self.gcp_project}.{self.dataset}.{model_name};" + + print("sql: ", sql) + print("gcp_project: ", self.gcp_project) + print("dataset: ", self.dataset) + print("gcp_conn_id: ", self.gcp_conn_id) + print("self.configuration: ", self.configuration) + print("self.location: ", self.location) hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, From 9081e6ad54dcf91baa8c6da5fa26d2fe5d917430 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 07:02:44 +0530 Subject: [PATCH 38/49] Adapt manifest DAG to use & adapt to the newer GCP conn secret that is to be updated soon --- dev/dags/cosmos_manifest_example.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/dags/cosmos_manifest_example.py b/dev/dags/cosmos_manifest_example.py index a96eff971..8e35208b8 100644 --- a/dev/dags/cosmos_manifest_example.py +++ b/dev/dags/cosmos_manifest_example.py @@ -74,7 +74,7 @@ def cosmos_manifest_example() -> None: gcp_gs_example = DbtTaskGroup( group_id="gcp_gs_example", project_config=ProjectConfig( - manifest_path="gs://cosmos-manifest-test/manifest.json", + manifest_path="gs://cosmos_remote_target/manifest.json", manifest_conn_id="gcp_gs_conn", # `manifest_conn_id` is optional. If not provided, the default connection ID `google_cloud_default` is used. project_name="jaffle_shop", From 2dccf84037ea79f3ce9375b0b8a78218106aca67 Mon Sep 17 00:00:00 2001 From: Tatiana Al-Chueyr Date: Wed, 2 Oct 2024 07:47:28 +0100 Subject: [PATCH 39/49] Release 1.7.0a1 --- cosmos/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cosmos/__init__.py b/cosmos/__init__.py index f76f18b7c..e82db83ea 100644 --- a/cosmos/__init__.py +++ b/cosmos/__init__.py @@ -5,7 +5,7 @@ Contains dags, task groups, and operators. """ -__version__ = "1.6.0" +__version__ = "1.7.0a1" from cosmos.airflow.dag import DbtDag From 7adeb994197e493663ba77dda39351d351805bf3 Mon Sep 17 00:00:00 2001 From: Tatiana Al-Chueyr Date: Wed, 2 Oct 2024 09:13:20 +0100 Subject: [PATCH 40/49] Retrigger GH actions --- cosmos/__init__.py | 1 + 1 file changed, 1 insertion(+) diff --git a/cosmos/__init__.py b/cosmos/__init__.py index e82db83ea..ad4c9ee35 100644 --- a/cosmos/__init__.py +++ b/cosmos/__init__.py @@ -5,6 +5,7 @@ Contains dags, task groups, and operators. """ + __version__ = "1.7.0a1" From 7e6de3044edde90e6757c161b40bfc27c9d0164a Mon Sep 17 00:00:00 2001 From: Tatiana Al-Chueyr Date: Wed, 2 Oct 2024 10:04:56 +0100 Subject: [PATCH 41/49] temporarily move out simple_dag_async.py --- dev/dags/simple_dag_async.py => simple_dag_async.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename dev/dags/simple_dag_async.py => simple_dag_async.py (100%) diff --git a/dev/dags/simple_dag_async.py b/simple_dag_async.py similarity index 100% rename from dev/dags/simple_dag_async.py rename to simple_dag_async.py From 16a87ea1299dbeac027846a2fb4be4914a03013d Mon Sep 17 00:00:00 2001 From: Tatiana Al-Chueyr Date: Wed, 2 Oct 2024 11:20:46 +0100 Subject: [PATCH 42/49] Fix CI issue --- cosmos/airflow/graph.py | 5 +++-- simple_dag_async.py => dags/simple_dag_async.py | 0 tests/test_example_dags.py | 8 +++++++- 3 files changed, 10 insertions(+), 3 deletions(-) rename simple_dag_async.py => dags/simple_dag_async.py (100%) diff --git a/cosmos/airflow/graph.py b/cosmos/airflow/graph.py index fafa2e467..7bed9ebcb 100644 --- a/cosmos/airflow/graph.py +++ b/cosmos/airflow/graph.py @@ -279,6 +279,8 @@ def _add_dbt_compile_task( compile_airflow_task = create_airflow_task(compile_task_metadata, dag, task_group=task_group) tasks_map[DBT_COMPILE_TASK_ID] = compile_airflow_task + # TODO: check within tasks_map.values() all the tasks that do not have upstream node + for node_id, node in nodes.items(): if not node.depends_on and node_id in tasks_map: tasks_map[DBT_COMPILE_TASK_ID] >> tasks_map[node_id] @@ -377,9 +379,8 @@ def build_airflow_graph( for leaf_node_id in leaves_ids: tasks_map[leaf_node_id] >> test_task - _add_dbt_compile_task(nodes, dag, execution_mode, task_args, tasks_map, task_group) - create_airflow_task_dependencies(nodes, tasks_map) + _add_dbt_compile_task(nodes, dag, execution_mode, task_args, tasks_map, task_group) def create_airflow_task_dependencies( diff --git a/simple_dag_async.py b/dags/simple_dag_async.py similarity index 100% rename from simple_dag_async.py rename to dags/simple_dag_async.py diff --git a/tests/test_example_dags.py b/tests/test_example_dags.py index 9aa66432d..70cb47398 100644 --- a/tests/test_example_dags.py +++ b/tests/test_example_dags.py @@ -98,4 +98,10 @@ def test_example_dag(session, dag_id: str): return dag_bag = get_dag_bag() dag = dag_bag.get_dag(dag_id) - test_utils.run_dag(dag) + + # This feature is available since Airflow 2.5 and we've backported it in Cosmos: + # https://airflow.apache.org/docs/apache-airflow/stable/release_notes.html#airflow-2-5-0-2022-12-02 + if AIRFLOW_VERSION >= Version("2.5"): + dag.test() + else: + test_utils.run_dag(dag) From 05db6a0cc75f50efc980bc290eda53347e2112d0 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 16:40:31 +0530 Subject: [PATCH 43/49] Fix dbt-compile dependency by using Airflow tasks instead of dbt nodes --- cosmos/airflow/graph.py | 9 ++++----- {dags => dev/dags}/simple_dag_async.py | 0 2 files changed, 4 insertions(+), 5 deletions(-) rename {dags => dev/dags}/simple_dag_async.py (100%) diff --git a/cosmos/airflow/graph.py b/cosmos/airflow/graph.py index 7bed9ebcb..f507b03ac 100644 --- a/cosmos/airflow/graph.py +++ b/cosmos/airflow/graph.py @@ -277,13 +277,12 @@ def _add_dbt_compile_task( extra_context={"dbt_dag_task_group_identifier": _get_dbt_dag_task_group_identifier(dag, task_group)}, ) compile_airflow_task = create_airflow_task(compile_task_metadata, dag, task_group=task_group) - tasks_map[DBT_COMPILE_TASK_ID] = compile_airflow_task - # TODO: check within tasks_map.values() all the tasks that do not have upstream node + for task_id, task in tasks_map.items(): + if not task.upstream_list: + compile_airflow_task >> task - for node_id, node in nodes.items(): - if not node.depends_on and node_id in tasks_map: - tasks_map[DBT_COMPILE_TASK_ID] >> tasks_map[node_id] + tasks_map[DBT_COMPILE_TASK_ID] = compile_airflow_task def _get_dbt_dag_task_group_identifier(dag: DAG, task_group: TaskGroup | None) -> str: diff --git a/dags/simple_dag_async.py b/dev/dags/simple_dag_async.py similarity index 100% rename from dags/simple_dag_async.py rename to dev/dags/simple_dag_async.py From 8fc4ae2fd5663491ab482efee60d9e2c638ee394 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 17:13:06 +0530 Subject: [PATCH 44/49] Apply suggestions from code review Co-authored-by: Tatiana Al-Chueyr --- .github/workflows/test.yml | 2 +- cosmos/operators/airflow_async.py | 7 ------- 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index baa535d4b..2cfe0e00d 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -2,7 +2,7 @@ name: test on: push: # Run on pushes to the default branch - branches: [main,poc-dbt-compile-task] + branches: [main] pull_request_target: # Also run on pull requests originated from forks branches: [main] diff --git a/cosmos/operators/airflow_async.py b/cosmos/operators/airflow_async.py index cb8838f97..a7f30a330 100644 --- a/cosmos/operators/airflow_async.py +++ b/cosmos/operators/airflow_async.py @@ -136,7 +136,6 @@ def get_remote_sql(self) -> str: relative_file_path = str(file_path).replace(project_dir_parent, "").lstrip("/") remote_model_path = f"{remote_target_path_str}/{dbt_dag_task_group_identifier}/compiled/{relative_file_path}" - print("remote_model_path: ", remote_model_path) object_storage_path = ObjectStoragePath(remote_model_path, conn_id=remote_target_path_conn_id) with object_storage_path.open() as fp: # type: ignore return fp.read() # type: ignore @@ -145,12 +144,6 @@ def drop_table_sql(self) -> None: model_name = self.extra_context["dbt_node_config"]["resource_name"] # type: ignore sql = f"DROP TABLE IF EXISTS {self.gcp_project}.{self.dataset}.{model_name};" - print("sql: ", sql) - print("gcp_project: ", self.gcp_project) - print("dataset: ", self.dataset) - print("gcp_conn_id: ", self.gcp_conn_id) - print("self.configuration: ", self.configuration) - print("self.location: ", self.location) hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, From ea5816bc47ba3892cf8c950a9fb41b7f4abef21b Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 2 Oct 2024 17:13:58 +0530 Subject: [PATCH 45/49] Apply suggestions from code review --- cosmos/operators/base.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/cosmos/operators/base.py b/cosmos/operators/base.py index 45f8bd173..ed7969ebd 100644 --- a/cosmos/operators/base.py +++ b/cosmos/operators/base.py @@ -110,7 +110,6 @@ def __init__( dbt_cmd_global_flags: list[str] | None = None, cache_dir: Path | None = None, extra_context: dict[str, Any] | None = None, - # configuration: dict[str, Any] | None = None, **kwargs: Any, ) -> None: self.project_dir = project_dir @@ -141,7 +140,6 @@ def __init__( self.cache_dir = cache_dir self.extra_context = extra_context or {} kwargs.pop("full_refresh", None) # usage of this param should be implemented in child classes - # kwargs["configuration"] = {} super().__init__(**kwargs) def get_env(self, context: Context) -> dict[str, str | bytes | os.PathLike[Any]]: From 85f86a45f5306985fdd4b8f0a5e286ee47eb714b Mon Sep 17 00:00:00 2001 From: pankajastro Date: Thu, 3 Oct 2024 14:31:20 +0530 Subject: [PATCH 46/49] Add install instruction --- docs/getting_started/execution-modes.rst | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docs/getting_started/execution-modes.rst b/docs/getting_started/execution-modes.rst index 0c1b2f54d..328461f4a 100644 --- a/docs/getting_started/execution-modes.rst +++ b/docs/getting_started/execution-modes.rst @@ -274,6 +274,13 @@ Note that currently, the ``airflow_async`` execution mode has the following limi 3. Only works for ``full_refresh`` models. There is pending work to support other modes. 4. Only Support for the Bigquery profile type 5. Users need to provide ProfileMapping parameter in ProfileConfig +6. It does not support dataset + +You can leverage async operator support by installing an additional dependency + +.. code:: bash + + astronomer-cosmos[dbt-bigquery, google] Example DAG: @@ -310,6 +317,8 @@ This causes the BigQuery trigger to attempt accessing parameters of the Task Ins TaskInstance.dag_id == self.task_instance.dag_id, AttributeError: 'NoneType' object has no attribute 'dag_id' + + .. _invocation_modes: Invocation Modes ================ From 402f82357465ac0144500b04aa5b59c3c40fa055 Mon Sep 17 00:00:00 2001 From: pankajastro Date: Thu, 3 Oct 2024 14:37:44 +0530 Subject: [PATCH 47/49] Add min airflow version in limitation --- docs/getting_started/execution-modes.rst | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/docs/getting_started/execution-modes.rst b/docs/getting_started/execution-modes.rst index 328461f4a..10f6cce67 100644 --- a/docs/getting_started/execution-modes.rst +++ b/docs/getting_started/execution-modes.rst @@ -269,12 +269,13 @@ deferrable operators and supplying to them those compiled SQLs. Note that currently, the ``airflow_async`` execution mode has the following limitations and is released as Experimental: -1. Only supports the ``dbt resource type`` models to be run asynchronously using Airflow deferrable operators. All other resources are executed synchronously using dbt commands as they are in the ``local`` execution mode. -2. Only supports BigQuery as the target database. If a profile target other than BigQuery is specified, Cosmos will error out saying that the target database is not supported with this execution mode. -3. Only works for ``full_refresh`` models. There is pending work to support other modes. -4. Only Support for the Bigquery profile type -5. Users need to provide ProfileMapping parameter in ProfileConfig -6. It does not support dataset +1. This feature only works when using Airflow 2.8 and above +2. Only supports the ``dbt resource type`` models to be run asynchronously using Airflow deferrable operators. All other resources are executed synchronously using dbt commands as they are in the ``local`` execution mode. +3. Only supports BigQuery as the target database. If a profile target other than BigQuery is specified, Cosmos will error out saying that the target database is not supported with this execution mode. +4. Only works for ``full_refresh`` models. There is pending work to support other modes. +5. Only Support for the Bigquery profile type +6. Users need to provide ProfileMapping parameter in ProfileConfig +7. It does not support dataset You can leverage async operator support by installing an additional dependency From 621a4de3ff4987bfe070333c7e35c24e50f36f61 Mon Sep 17 00:00:00 2001 From: pankajastro Date: Thu, 3 Oct 2024 15:27:54 +0530 Subject: [PATCH 48/49] Ignore Async DAG for dbt <=1.5 --- scripts/test/integration-sqlite-setup.sh | 2 +- tests/test_example_dags.py | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/scripts/test/integration-sqlite-setup.sh b/scripts/test/integration-sqlite-setup.sh index c7f547ce9..b8bce035c 100644 --- a/scripts/test/integration-sqlite-setup.sh +++ b/scripts/test/integration-sqlite-setup.sh @@ -1,4 +1,4 @@ pip uninstall -y dbt-core dbt-sqlite openlineage-airflow openlineage-integration-common; \ rm -rf airflow.*; \ airflow db init; \ -pip install 'dbt-core==1.4' 'dbt-sqlite<=1.4' 'dbt-databricks<=1.4' 'dbt-postgres<=1.4' 'dbt-bigquery<=1.4' +pip install 'dbt-core==1.4' 'dbt-sqlite<=1.4' 'dbt-databricks<=1.4' 'dbt-postgres<=1.4' diff --git a/tests/test_example_dags.py b/tests/test_example_dags.py index 70cb47398..b02d0adaf 100644 --- a/tests/test_example_dags.py +++ b/tests/test_example_dags.py @@ -68,6 +68,10 @@ def get_dag_bag() -> DagBag: print(f"Adding {dagfile} to .airflowignore") file.writelines([f"{dagfile}\n"]) + # Ignore Async DAG for dbt 1.4 + if DBT_VERSION <= Version("1.5.0"): + file.writelines(["simple_dag_async.py\n"]) + # The dbt sqlite adapter is only available until dbt 1.4 if DBT_VERSION >= Version("1.5.0"): file.writelines(["example_cosmos_sources.py\n"]) From a0cb147f1ad3776838cccca0d9ffd696a9bb5cb9 Mon Sep 17 00:00:00 2001 From: pankajastro Date: Thu, 3 Oct 2024 16:07:00 +0530 Subject: [PATCH 49/49] Ignore Async DAG for dbt <=1.5 --- tests/test_example_dags.py | 2 +- tests/test_example_dags_no_connections.py | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/test_example_dags.py b/tests/test_example_dags.py index b02d0adaf..6c7e98802 100644 --- a/tests/test_example_dags.py +++ b/tests/test_example_dags.py @@ -68,7 +68,7 @@ def get_dag_bag() -> DagBag: print(f"Adding {dagfile} to .airflowignore") file.writelines([f"{dagfile}\n"]) - # Ignore Async DAG for dbt 1.4 + # Ignore Async DAG for dbt <=1.5 if DBT_VERSION <= Version("1.5.0"): file.writelines(["simple_dag_async.py\n"]) diff --git a/tests/test_example_dags_no_connections.py b/tests/test_example_dags_no_connections.py index 0cc560ecc..3a43a644c 100644 --- a/tests/test_example_dags_no_connections.py +++ b/tests/test_example_dags_no_connections.py @@ -43,6 +43,10 @@ def get_dag_bag() -> DagBag: print(f"Adding {dagfile} to .airflowignore") file.writelines([f"{dagfile}\n"]) + # Ignore Async DAG for dbt <=1.5 + if DBT_VERSION <= Version("1.5.0"): + file.writelines(["simple_dag_async.py\n"]) + if DBT_VERSION >= Version("1.5.0"): file.writelines(["example_cosmos_sources.py\n"]) if DBT_VERSION < Version("1.6.0"):