From 0fb5be61c682f0358cbdecee8c5149482f88e26d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 18 Oct 2021 09:49:24 +0200 Subject: [PATCH 01/79] Decouple hook executor from the operator, add Cloud Build execution and adjust testing --- airflow_dbt/hooks/dbt_hook.py | 223 ++++++++++++++------------ airflow_dbt/operators/dbt_operator.py | 102 ++++++++---- setup.py | 3 + tests/hooks/test_dbt_hook.py | 65 +++----- tests/operators/test_dbt_operator.py | 39 +++-- 5 files changed, 227 insertions(+), 205 deletions(-) diff --git a/airflow_dbt/hooks/dbt_hook.py b/airflow_dbt/hooks/dbt_hook.py index 0b4caf0..bf2c3fc 100644 --- a/airflow_dbt/hooks/dbt_hook.py +++ b/airflow_dbt/hooks/dbt_hook.py @@ -1,142 +1,157 @@ from __future__ import print_function -import os -import signal -import subprocess + import json -from airflow.exceptions import AirflowException +from abc import ABC, abstractmethod +from typing import Any, Dict, List, Union + from airflow.hooks.base_hook import BaseHook +from airflow.hooks.subprocess import SubprocessHook -class DbtCliHook(BaseHook): +class DbtBaseHook(BaseHook, ABC): """ Simple wrapper around the dbt CLI. - :param profiles_dir: If set, passed as the `--profiles-dir` argument to the `dbt` command - :type profiles_dir: str - :param target: If set, passed as the `--target` argument to the `dbt` command :type dir: str :param dir: The directory to run the CLI in - :type vars: str - :param vars: If set, passed as the `--vars` argument to the `dbt` command - :type vars: dict - :param full_refresh: If `True`, will fully-refresh incremental models. - :type full_refresh: bool - :param models: If set, passed as the `--models` argument to the `dbt` command - :type models: str - :param warn_error: If `True`, treat warnings as errors. - :type warn_error: bool - :param exclude: If set, passed as the `--exclude` argument to the `dbt` command - :type exclude: str - :param select: If set, passed as the `--select` argument to the `dbt` command - :type select: str - :param dbt_bin: The `dbt` CLI. Defaults to `dbt`, so assumes it's on your `PATH` + :type env: dict + :param env: If set, passed to the dbt executor + :param dbt_bin: The `dbt` CLI. Defaults to `dbt`, so assumes it's on your + `PATH` :type dbt_bin: str - :param output_encoding: Output encoding of bash command. Defaults to utf-8 - :type output_encoding: str - :param verbose: The operator will log verbosely to the Airflow logs - :type verbose: bool """ - def __init__(self, - profiles_dir=None, - target=None, - dir='.', - vars=None, - full_refresh=False, - data=False, - schema=False, - models=None, - exclude=None, - select=None, - dbt_bin='dbt', - output_encoding='utf-8', - verbose=True, - warn_error=False): - self.profiles_dir = profiles_dir + def __init__(self, dir: str = '.', env: Dict = None, dbt_bin='dbt'): + super().__init__() self.dir = dir - self.target = target - self.vars = vars - self.full_refresh = full_refresh - self.data = data - self.schema = schema - self.models = models - self.exclude = exclude - self.select = select + self.env = env if env is not None else {} self.dbt_bin = dbt_bin - self.verbose = verbose - self.warn_error = warn_error - self.output_encoding = output_encoding - - def _dump_vars(self): - # The dbt `vars` parameter is defined using YAML. Unfortunately the standard YAML library - # for Python isn't very good and I couldn't find an easy way to have it formatted - # correctly. However, as YAML is a super-set of JSON, this works just fine. - return json.dumps(self.vars) - def run_cli(self, *command): + def generate_dbt_cli_command( + self, + base_command: str, + profiles_dir: str = None, + target: str = None, + vars: Dict[str, str] = None, + full_refresh: bool = False, + data: bool = False, + schema: bool = False, + models: str = None, + exclude: str = None, + select: str = None, + warn_error: bool = False, + ) -> List[str]: """ - Run the dbt cli - - :param command: The dbt command to run - :type command: str + Generate the command that will be run based on class properties, + presets and dbt commands + + :param base_command: The dbt sub-command to run + :type base_command: str + :param profiles_dir: If set, passed as the `--profiles-dir` argument to + the `dbt` command + :type profiles_dir: str + :param target: If set, passed as the `--target` argument to the `dbt` + command + :type vars: Union[str, dict] + :param vars: If set, passed as the `--vars` argument to the `dbt` + command + :param full_refresh: If `True`, will fully-refresh incremental models. + :type full_refresh: bool + :param data: + :type data: bool + :param schema: + :type schema: bool + :param models: If set, passed as the `--models` argument to the `dbt` + command + :type models: str + :param warn_error: If `True`, treat warnings as errors. + :type warn_error: bool + :param exclude: If set, passed as the `--exclude` argument to the `dbt` + command + :type exclude: str + :param select: If set, passed as the `--select` argument to the `dbt` + command + :type select: str """ + dbt_cmd = [self.dbt_bin, base_command] - dbt_cmd = [self.dbt_bin, *command] - - if self.profiles_dir is not None: - dbt_cmd.extend(['--profiles-dir', self.profiles_dir]) + if profiles_dir is not None: + dbt_cmd.extend(['--profiles-dir', profiles_dir]) - if self.target is not None: - dbt_cmd.extend(['--target', self.target]) + if target is not None: + dbt_cmd.extend(['--target', target]) - if self.vars is not None: - dbt_cmd.extend(['--vars', self._dump_vars()]) + if vars is not None: + dbt_cmd.extend(['--vars', json.dumps(vars)]) - if self.data: + if data: dbt_cmd.extend(['--data']) - if self.schema: + if schema: dbt_cmd.extend(['--schema']) - if self.models is not None: - dbt_cmd.extend(['--models', self.models]) + if models is not None: + dbt_cmd.extend(['--models', models]) - if self.exclude is not None: - dbt_cmd.extend(['--exclude', self.exclude]) + if exclude is not None: + dbt_cmd.extend(['--exclude', self]) - if self.select is not None: - dbt_cmd.extend(['--select', self.select]) + if select is not None: + dbt_cmd.extend(['--select', select]) - if self.full_refresh: + if full_refresh: dbt_cmd.extend(['--full-refresh']) - if self.warn_error: + if warn_error: dbt_cmd.insert(1, '--warn-error') - if self.verbose: - self.log.info(" ".join(dbt_cmd)) + return dbt_cmd + + @abstractmethod + def run_dbt(self, dbt_cmd: Union[str, List[str]]): + """Run the dbt command""" + + +class DbtCliHook(DbtBaseHook): + """ + Run the dbt command in the same airflow worker the task is being run. + This requires the `dbt` python package to be installed in it first. Also + the dbt_bin path might not be set in the `PATH` variable, so it could be + necessary to set it in the constructor. + + :type dir: str + :param dir: The directory to run the CLI in + :type env: dict + :param env: If set, passed to the dbt executor + :param dbt_bin: The `dbt` CLI. Defaults to `dbt`, so assumes it's on your + `PATH` + :type dbt_bin: str + """ + + def __init__(self, dir: str = '.', env: Dict = None, dbt_bin='dbt'): + self.sp = SubprocessHook() + super().__init__(dir=dir, env=env, dbt_bin=dbt_bin) - sp = subprocess.Popen( - dbt_cmd, - stdout=subprocess.PIPE, - stderr=subprocess.STDOUT, + def get_conn(self) -> Any: + """ + Return the subprocess connection, which isn't implemented, just for + conformity + """ + return self.sp.get_conn() + + def run_dbt(self, dbt_cmd: Union[str, List[str]]): + """ + Run the dbt cli + + :param dbt_cmd: The dbt whole command to run + :type dbt_cmd: List[str] + """ + self.sp.run_command( + command=dbt_cmd, + env=self.env, cwd=self.dir, - close_fds=True) - self.sp = sp - self.log.info("Output:") - line = '' - for line in iter(sp.stdout.readline, b''): - line = line.decode(self.output_encoding).rstrip() - self.log.info(line) - sp.wait() - self.log.info( - "Command exited with return code %s", - sp.returncode ) - if sp.returncode: - raise AirflowException("dbt command failed") - def on_kill(self): - self.log.info('Sending SIGTERM signal to dbt command') - os.killpg(os.getpgid(self.sp.pid), signal.SIGTERM) + """Kill the open subprocess if the task gets killed by Airflow""" + self.sp.send_sigterm() diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 6233d8d..7be49e9 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -1,7 +1,10 @@ -from airflow_dbt.hooks.dbt_hook import DbtCliHook +from typing import Any + from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults +from airflow_dbt.hooks.dbt_hook import DbtCliHook + class DbtBaseOperator(BaseOperator): """ @@ -30,6 +33,16 @@ class DbtBaseOperator(BaseOperator): :type dbt_bin: str :param verbose: The operator will log verbosely to the Airflow logs :type verbose: bool + :param dbt_hook: The dbt hook to use as executor. For now the + implemented ones are: DbtCliHook, DbtCloudBuildHook. It should be an + instance of one of those, or another that inherits from DbtBaseHook. If + not provided by default a DbtCliHook will be instantiated with the + provided params + :type dbt_hook: DbtBaseHook + :param base_command: The dbt sub command to run, for example for `dbt + run` the base_command will be `run`. If any other flag not + contemplated must be included it can also be added to this string + :type base_command: str """ ui_color = '#d6522a' @@ -51,6 +64,8 @@ def __init__(self, full_refresh=False, data=False, schema=False, + dbt_hook=None, + base_command=None, *args, **kwargs): super(DbtBaseOperator, self).__init__(*args, **kwargs) @@ -68,13 +83,15 @@ def __init__(self, self.dbt_bin = dbt_bin self.verbose = verbose self.warn_error = warn_error - self.create_hook() + self.base_command = base_command + self.hook = dbt_hook if dbt_hook is not None else DbtCliHook() - def create_hook(self): - self.hook = DbtCliHook( + def execute(self, context: Any): + """Runs the provided command in the provided execution environment""" + dbt_cli_command = self.hook.generate_dbt_cli_command( + base_command=self.base_command, profiles_dir=self.profiles_dir, target=self.target, - dir=self.dir, vars=self.vars, full_refresh=self.full_refresh, data=self.data, @@ -82,63 +99,78 @@ def create_hook(self): models=self.models, exclude=self.exclude, select=self.select, - dbt_bin=self.dbt_bin, - verbose=self.verbose, - warn_error=self.warn_error) - - return self.hook + warn_error=self.warn_error, + ) + self.hook.run_dbt(dbt_cli_command) class DbtRunOperator(DbtBaseOperator): @apply_defaults def __init__(self, profiles_dir=None, target=None, *args, **kwargs): - super(DbtRunOperator, self).__init__(profiles_dir=profiles_dir, target=target, *args, **kwargs) - - def execute(self, context): - self.create_hook().run_cli('run') + super().__init__( + profiles_dir=profiles_dir, + target=target, + base_command='run', + *args, + **kwargs + ) class DbtTestOperator(DbtBaseOperator): @apply_defaults def __init__(self, profiles_dir=None, target=None, *args, **kwargs): - super(DbtTestOperator, self).__init__(profiles_dir=profiles_dir, target=target, *args, **kwargs) - - def execute(self, context): - self.create_hook().run_cli('test') + super().__init__( + profiles_dir=profiles_dir, + target=target, + base_command='test', + *args, + **kwargs + ) class DbtDocsGenerateOperator(DbtBaseOperator): @apply_defaults def __init__(self, profiles_dir=None, target=None, *args, **kwargs): - super(DbtDocsGenerateOperator, self).__init__(profiles_dir=profiles_dir, target=target, *args, - **kwargs) - - def execute(self, context): - self.create_hook().run_cli('docs', 'generate') + super().__init__( + profiles_dir=profiles_dir, + target=target, + base_command='docs generate', + *args, + **kwargs + ) class DbtSnapshotOperator(DbtBaseOperator): @apply_defaults def __init__(self, profiles_dir=None, target=None, *args, **kwargs): - super(DbtSnapshotOperator, self).__init__(profiles_dir=profiles_dir, target=target, *args, **kwargs) - - def execute(self, context): - self.create_hook().run_cli('snapshot') + super().__init__( + profiles_dir=profiles_dir, + target=target, + base_command='snapshot', + *args, + **kwargs + ) class DbtSeedOperator(DbtBaseOperator): @apply_defaults def __init__(self, profiles_dir=None, target=None, *args, **kwargs): - super(DbtSeedOperator, self).__init__(profiles_dir=profiles_dir, target=target, *args, **kwargs) - - def execute(self, context): - self.create_hook().run_cli('seed') + super().__init__( + profiles_dir=profiles_dir, + target=target, + base_command='seed', + *args, + **kwargs + ) class DbtDepsOperator(DbtBaseOperator): @apply_defaults def __init__(self, profiles_dir=None, target=None, *args, **kwargs): - super(DbtDepsOperator, self).__init__(profiles_dir=profiles_dir, target=target, *args, **kwargs) - - def execute(self, context): - self.create_hook().run_cli('deps') + super().__init__( + profiles_dir=profiles_dir, + target=target, + base_command='deps', + *args, + **kwargs + ) diff --git a/setup.py b/setup.py index 7d3b2f8..325e51b 100644 --- a/setup.py +++ b/setup.py @@ -78,4 +78,7 @@ def run(self): cmdclass={ 'upload': UploadCommand, }, + extras_require={ + 'google': 'apache-airflow-providers-google==5.0.0' + }, ) diff --git a/tests/hooks/test_dbt_hook.py b/tests/hooks/test_dbt_hook.py index 4dd39ed..e1990b8 100644 --- a/tests/hooks/test_dbt_hook.py +++ b/tests/hooks/test_dbt_hook.py @@ -1,54 +1,27 @@ -from unittest import TestCase -from unittest import mock -import subprocess -from airflow_dbt.hooks.dbt_hook import DbtCliHook +from unittest import TestCase, mock +from airflow.hooks.subprocess import SubprocessHook -class TestDbtHook(TestCase): +from airflow_dbt.hooks.dbt_hook import DbtCliHook - @mock.patch('subprocess.Popen') - def test_sub_commands(self, mock_subproc_popen): - mock_subproc_popen.return_value \ - .communicate.return_value = ('output', 'error') - mock_subproc_popen.return_value.returncode = 0 - mock_subproc_popen.return_value \ - .stdout.readline.side_effect = [b"placeholder"] +class TestDbtHook(TestCase): + @mock.patch.object(SubprocessHook, 'run_command') + def test_sub_commands(self, mock_run_command): hook = DbtCliHook() - hook.run_cli('docs', 'generate') - - mock_subproc_popen.assert_called_once_with( - [ - 'dbt', - 'docs', - 'generate' - ], - close_fds=True, + hook.run_dbt(['dbt', 'docs', 'generate']) + mock_run_command.assert_called_once_with( + command=['dbt', 'docs', 'generate'], + env={}, cwd='.', - stdout=subprocess.PIPE, - stderr=subprocess.STDOUT - ) - - @mock.patch('subprocess.Popen') - def test_vars(self, mock_subproc_popen): - mock_subproc_popen.return_value \ - .communicate.return_value = ('output', 'error') - mock_subproc_popen.return_value.returncode = 0 - mock_subproc_popen.return_value \ - .stdout.readline.side_effect = [b"placeholder"] + ) - hook = DbtCliHook(vars={"foo": "bar", "baz": "true"}) - hook.run_cli('run') + def test_vars(self): + hook = DbtCliHook() + generated_command = hook.generate_dbt_cli_command( + 'run', + vars={"foo": "bar", "baz": "true"} + ) - mock_subproc_popen.assert_called_once_with( - [ - 'dbt', - 'run', - '--vars', - '{"foo": "bar", "baz": "true"}' - ], - close_fds=True, - cwd='.', - stdout=subprocess.PIPE, - stderr=subprocess.STDOUT - ) + assert generated_command == ['dbt', 'run', '--vars', + '{"foo": "bar", "baz": "true"}'] diff --git a/tests/operators/test_dbt_operator.py b/tests/operators/test_dbt_operator.py index 8ce2c5f..236396d 100644 --- a/tests/operators/test_dbt_operator.py +++ b/tests/operators/test_dbt_operator.py @@ -1,17 +1,16 @@ import datetime from unittest import TestCase, mock + from airflow import DAG, configuration + from airflow_dbt.hooks.dbt_hook import DbtCliHook from airflow_dbt.operators.dbt_operator import ( - DbtSeedOperator, - DbtSnapshotOperator, - DbtRunOperator, + DbtDepsOperator, DbtRunOperator, DbtSeedOperator, DbtSnapshotOperator, DbtTestOperator, - DbtDepsOperator ) -class TestDbtOperator(TestCase): +class TestDbtCliOperator(TestCase): def setUp(self): configuration.conf.load_test_config() args = { @@ -20,47 +19,47 @@ def setUp(self): } self.dag = DAG('test_dag_id', default_args=args) - @mock.patch.object(DbtCliHook, 'run_cli') - def test_dbt_run(self, mock_run_cli): + @mock.patch.object(DbtCliHook, 'run_dbt') + def test_dbt_run(self, mock_run_dbt): operator = DbtRunOperator( task_id='run', dag=self.dag ) operator.execute(None) - mock_run_cli.assert_called_once_with('run') + mock_run_dbt.assert_called_once_with(['dbt', 'run']) - @mock.patch.object(DbtCliHook, 'run_cli') - def test_dbt_test(self, mock_run_cli): + @mock.patch.object(DbtCliHook, 'run_dbt') + def test_dbt_test(self, mock_run_dbt): operator = DbtTestOperator( task_id='test', dag=self.dag ) operator.execute(None) - mock_run_cli.assert_called_once_with('test') + mock_run_dbt.assert_called_once_with(['dbt', 'test']) - @mock.patch.object(DbtCliHook, 'run_cli') - def test_dbt_snapshot(self, mock_run_cli): + @mock.patch.object(DbtCliHook, 'run_dbt') + def test_dbt_snapshot(self, mock_run_dbt): operator = DbtSnapshotOperator( task_id='snapshot', dag=self.dag ) operator.execute(None) - mock_run_cli.assert_called_once_with('snapshot') + mock_run_dbt.assert_called_once_with(['dbt', 'snapshot']) - @mock.patch.object(DbtCliHook, 'run_cli') - def test_dbt_seed(self, mock_run_cli): + @mock.patch.object(DbtCliHook, 'run_dbt') + def test_dbt_seed(self, mock_run_dbt): operator = DbtSeedOperator( task_id='seed', dag=self.dag ) operator.execute(None) - mock_run_cli.assert_called_once_with('seed') + mock_run_dbt.assert_called_once_with(['dbt', 'seed']) - @mock.patch.object(DbtCliHook, 'run_cli') - def test_dbt_deps(self, mock_run_cli): + @mock.patch.object(DbtCliHook, 'run_dbt') + def test_dbt_deps(self, mock_run_dbt): operator = DbtDepsOperator( task_id='deps', dag=self.dag ) operator.execute(None) - mock_run_cli.assert_called_once_with('deps') + mock_run_dbt.assert_called_once_with(['dbt', 'deps']) From e53b7442f1c548b60b569f30067706fa6ca42cd9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 18 Oct 2021 13:01:38 +0200 Subject: [PATCH 02/79] Add hook to run in google cloud build --- airflow_dbt/hooks/dbt_google_hook.py | 161 +++++++++++++++++++++++++++ 1 file changed, 161 insertions(+) create mode 100644 airflow_dbt/hooks/dbt_google_hook.py diff --git a/airflow_dbt/hooks/dbt_google_hook.py b/airflow_dbt/hooks/dbt_google_hook.py new file mode 100644 index 0000000..ff63527 --- /dev/null +++ b/airflow_dbt/hooks/dbt_google_hook.py @@ -0,0 +1,161 @@ +import logging +import os +import tarfile +from tempfile import NamedTemporaryFile +from typing import Any, Dict, List +from uuid import uuid4 + +from airflow.exceptions import AirflowException +from airflow.providers.google.cloud.hooks.cloud_build import CloudBuildHook +from airflow.providers.google.cloud.hooks.gcs import ( + GCSHook, _parse_gcs_url, gcs_object_is_directory, +) + +from hooks.dbt_hook import DbtBaseHook + + +class DbtCloudBuildHook(DbtBaseHook): + """ + Runs the dbt command in a Cloud Build job in GCP + + :type dir: str + :param dir: The directory containing the DBT files. The logic is, if this + is a GCS blob compressed in tar.gz then it will be used in the build + process without re-uploading. Otherwise we expect "dir" to point to a + local path to be uploaded to the GCS prefix set in + "gcs_staging_location". + :type env: dict + :param env: If set, passed to the dbt executor + :param dbt_bin: The `dbt` CLI. Defaults to `dbt`, so assumes it's on your + `PATH` + :type dbt_bin: str + + :param project_id: GCP Project ID as stated in the console + :type project_id: str + :param timeout: Default is set in Cloud Build itself as ten minutes. A + duration in seconds with up to nine fractional digits, terminated by + 's'. Example: "3.5s" + :type timeout: str + :param wait: Waits for the cloud build process to finish. That is waiting + for the DBT command to finish running or run asynchronously + :type wait: bool + :param gcp_conn_id: The connection ID to use when fetching connection info. + :type gcp_conn_id: str + :param gcs_staging_location: Where to store the sources to be fetch later + by the cloud build job. It should be the GCS url for a folder. For + example: `gs://my-bucket/stored. A sub-folder will be generated to + avoid collision between possible different concurrent runs. + :param gcs_staging_location: str + :param dbt_version: the DBT version to be fetched from dockerhub. Defaults + to '0.21.0' + :type dbt_version: str + """ + + def __init__( + self, + project_id: str, + dir: str = '.', + gcs_staging_location: str = None, + timeout: str = None, + wait: bool = True, + gcp_conn_id: str = "google_cloud_default", + dbt_version: str = '0.21.0', + env: Dict = None, + dbt_bin='dbt', + ): + if dir is not None and gcs_staging_location is not None: + logging.info(f'Files in "{dir}" will be uploaded to GCS with the ' + f'prefix "{gcs_staging_location}"') + # check the destination is a gcs directory and extract bucket and + # folder + if not gcs_object_is_directory(gcs_staging_location): + raise ValueError( + f'The provided "gcs_sources_location": "' + f'{gcs_staging_location}"' + f' is not a valid gcs folder' + ) + slb, slp = _parse_gcs_url(gcs_staging_location) + # we have provided something similar to 'gs:///' + self.gcs_staging_bucket = slb + self.gcs_staging_blob = f'{slp}dbt_staging_{uuid4().hex}.tar.gz' + self.upload = True + + elif dir is not None and gcs_staging_location is None: + logging.info('Files in the "{dir}" blob will be used') + staging_bucket, staging_blob = _parse_gcs_url(dir) + if not staging_blob.endswith('.tar.gz'): + raise AirflowException( + f'The provided blob "{dir}" to a compressed file does not '+ + f'have the right extension ".tar.gz' + ) + self.gcs_staging_bucket = staging_bucket + self.gcs_staging_blob = staging_blob + self.upload = False + + self.dbt_version = dbt_version + self.cloud_build_hook = CloudBuildHook(gcp_conn_id=gcp_conn_id) + self.gcp_conn_id = gcp_conn_id + self.project_id = project_id + self.timeout = timeout + self.wait = wait + + super().__init__(dir=dir, env=env, dbt_bin=dbt_bin) + + def get_conn(self) -> Any: + """Returns the cloud build connection, which is a gcp connection""" + return self.cloud_build_hook.get_conn() + + def upload_dbt_sources(self) -> None: + """Upload sources from local to a staging location""" + gcs_hook = GCSHook(gcp_conn_id=self.gcp_conn_id) + with \ + NamedTemporaryFile() as compressed_file, \ + tarfile.open(compressed_file.name, "w:gz") as tar: + tar.add(self.dir, arcname=os.path.basename(self.dir)) + gcs_hook.upload( + bucket_name=self.gcs_staging_bucket, + object_name=self.gcs_staging_blob, + filename=compressed_file.name, + ) + + def run_dbt(self, dbt_cmd: List[str]): + """ + Run the dbt cli + + :param dbt_cmd: The dbt whole command to run + :type dbt_cmd: List[str] + """ + """See: https://cloud.google.com/cloud-build/docs/api/reference/rest + /v1/projects.builds""" + + if self.upload: + self.upload_dbt_sources() + + results = self.cloud_build_hook.create_build( + build={ + 'steps': [{ + 'name': f'fishtownanalytics/dbt:{self.dbt_version}', + 'entrypoint': '/bin/sh', + 'args': ['-c', *dbt_cmd], + 'env': [f'{k}={v}' for k, v in self.env.items()] + }], + 'source': { + 'storageSource': { + "bucket": self.gcs_staging_bucket, + "object": self.gcs_staging_blob, + } + } + }, + project_id=self.project_id, + wait=True, + timeout=self.timeout, + metadata=self.env, + ) + logging.info( + f'Triggered build {results["id"]}\nYou can find the logs at ' + f'{results["logUrl"]}' + ) + + def on_kill(self): + """Stopping the build is not implemented until google providers v6""" + raise NotImplementedError From b922f21880453e331fdb1de663b282e242569652 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 18 Oct 2021 13:01:59 +0200 Subject: [PATCH 03/79] Fix base operator --- airflow_dbt/operators/dbt_operator.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 7be49e9..f4f2eaa 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -84,7 +84,10 @@ def __init__(self, self.verbose = verbose self.warn_error = warn_error self.base_command = base_command - self.hook = dbt_hook if dbt_hook is not None else DbtCliHook() + self.hook = dbt_hook if dbt_hook is not None else DbtCliHook( + dir=dir, + dbt_bin=dbt_bin + ) def execute(self, context: Any): """Runs the provided command in the provided execution environment""" From f2fb0188a14559ab49654c353f874e60700193c7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 18 Oct 2021 13:02:23 +0200 Subject: [PATCH 04/79] Test GCP Cloud Build hook --- tests/operators/test_dbt_operator.py | 45 ++++++++++++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/tests/operators/test_dbt_operator.py b/tests/operators/test_dbt_operator.py index 236396d..bb4dc31 100644 --- a/tests/operators/test_dbt_operator.py +++ b/tests/operators/test_dbt_operator.py @@ -1,8 +1,10 @@ import datetime from unittest import TestCase, mock +from unittest.mock import patch from airflow import DAG, configuration +from airflow_dbt.hooks.dbt_google_hook import DbtCloudBuildHook from airflow_dbt.hooks.dbt_hook import DbtCliHook from airflow_dbt.operators.dbt_operator import ( DbtDepsOperator, DbtRunOperator, DbtSeedOperator, DbtSnapshotOperator, @@ -63,3 +65,46 @@ def test_dbt_deps(self, mock_run_dbt): ) operator.execute(None) mock_run_dbt.assert_called_once_with(['dbt', 'deps']) + + +class TestDbtRunWithCloudBuild(TestCase): + def setUp(self): + configuration.conf.load_test_config() + args = { + 'owner': 'airflow', + 'start_date': datetime.datetime(2020, 2, 27) + } + self.dag = DAG('test_dag_id', default_args=args) + + @patch('airflow_dbt.hooks.dbt_google_hook.CloudBuildHook') + @patch('airflow_dbt.hooks.dbt_google_hook.GCSHook') + def test_dbt_deps(self, MockLocalCloudBuildHook, MockGCSHook): + operator = DbtRunOperator( + task_id='test_dbt_run_on_cloud_build', + dbt_hook=DbtCloudBuildHook( + project_id='my-project-id', + gcp_conn_id='my_conn_id', + gcs_staging_location='gs://my-bucket/certain-folder/' + ), + dag=self.dag + ) + operator.execute(None) + MockLocalCloudBuildHook.assert_called_once_with(gcp_conn_id='my_conn_id') + MockGCSHook.assert_called_once_with(gcp_conn_id='my_conn_id') + MockGCSHook().upload.assert_called_once() + + +{ + 'steps': [{ + 'name': 'fishtownanalytics/dbt:0.21.0', 'entrypoint': '/bin/sh', + 'args': ['-c', 'dbt', 'run'], 'env': [] + }], + 'source': { + 'storageSource': { + 'bucket': 'my-bucket', + 'object': + 'certain-folder/dbt_staging_c5013d9965b54386bcf84ab76e42c848' + '.tar.gz' + } + } +} From ad837eef8a9c74f043deafe79850636401770a00 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 18 Oct 2021 14:44:30 +0200 Subject: [PATCH 05/79] Fix cloudbuild logic --- airflow_dbt/hooks/dbt_google_hook.py | 54 ++++++++++------------------ 1 file changed, 19 insertions(+), 35 deletions(-) diff --git a/airflow_dbt/hooks/dbt_google_hook.py b/airflow_dbt/hooks/dbt_google_hook.py index ff63527..5babc41 100644 --- a/airflow_dbt/hooks/dbt_google_hook.py +++ b/airflow_dbt/hooks/dbt_google_hook.py @@ -19,11 +19,8 @@ class DbtCloudBuildHook(DbtBaseHook): Runs the dbt command in a Cloud Build job in GCP :type dir: str - :param dir: The directory containing the DBT files. The logic is, if this - is a GCS blob compressed in tar.gz then it will be used in the build - process without re-uploading. Otherwise we expect "dir" to point to a - local path to be uploaded to the GCS prefix set in - "gcs_staging_location". + :param dir: Optional, if set the process considers that sources must be + uploaded prior to running the DBT job :type env: dict :param env: If set, passed to the dbt executor :param dbt_bin: The `dbt` CLI. Defaults to `dbt`, so assumes it's on your @@ -54,7 +51,7 @@ class DbtCloudBuildHook(DbtBaseHook): def __init__( self, project_id: str, - dir: str = '.', + dir: str = None, gcs_staging_location: str = None, timeout: str = None, wait: bool = True, @@ -63,34 +60,18 @@ def __init__( env: Dict = None, dbt_bin='dbt', ): - if dir is not None and gcs_staging_location is not None: - logging.info(f'Files in "{dir}" will be uploaded to GCS with the ' - f'prefix "{gcs_staging_location}"') - # check the destination is a gcs directory and extract bucket and - # folder - if not gcs_object_is_directory(gcs_staging_location): - raise ValueError( - f'The provided "gcs_sources_location": "' - f'{gcs_staging_location}"' - f' is not a valid gcs folder' - ) - slb, slp = _parse_gcs_url(gcs_staging_location) - # we have provided something similar to 'gs:///' - self.gcs_staging_bucket = slb - self.gcs_staging_blob = f'{slp}dbt_staging_{uuid4().hex}.tar.gz' - self.upload = True - - elif dir is not None and gcs_staging_location is None: - logging.info('Files in the "{dir}" blob will be used') - staging_bucket, staging_blob = _parse_gcs_url(dir) - if not staging_blob.endswith('.tar.gz'): - raise AirflowException( - f'The provided blob "{dir}" to a compressed file does not '+ - f'have the right extension ".tar.gz' + logging.info(f'Files in "{dir}" will be uploaded to GCS with the ' + f'prefix "{gcs_staging_location}"') + staging_bucket, staging_blob = _parse_gcs_url(gcs_staging_location) + # we have provided something similar to + # 'gs:///' + if not staging_blob.endswith('.tar.gz'): + raise AirflowException( + f'The provided blob "{staging_blob}" to a compressed file does not ' + + f'have the right extension ".tar.gz' ) - self.gcs_staging_bucket = staging_bucket - self.gcs_staging_blob = staging_blob - self.upload = False + self.gcs_staging_bucket = staging_bucket + self.gcs_staging_blob = staging_blob self.dbt_version = dbt_version self.cloud_build_hook = CloudBuildHook(gcp_conn_id=gcp_conn_id) @@ -128,7 +109,10 @@ def run_dbt(self, dbt_cmd: List[str]): """See: https://cloud.google.com/cloud-build/docs/api/reference/rest /v1/projects.builds""" - if self.upload: + # if we indicate that the sources are in a local directory by setting + # the "dir" pointing to a local path, then those sources will be + # uploaded to the expected blob + if self.dir is not None: self.upload_dbt_sources() results = self.cloud_build_hook.create_build( @@ -152,7 +136,7 @@ def run_dbt(self, dbt_cmd: List[str]): metadata=self.env, ) logging.info( - f'Triggered build {results["id"]}\nYou can find the logs at ' + f'Triggered build {results["id"]}. You can find the logs at ' f'{results["logUrl"]}' ) From 543d744553e5b219b79cbfd9d034e2db03dbc3f8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 18 Oct 2021 15:54:42 +0200 Subject: [PATCH 06/79] Clean unused imports and fix path to upload dbt sources in GCS --- airflow_dbt/hooks/dbt_google_hook.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/airflow_dbt/hooks/dbt_google_hook.py b/airflow_dbt/hooks/dbt_google_hook.py index 5babc41..6abbe11 100644 --- a/airflow_dbt/hooks/dbt_google_hook.py +++ b/airflow_dbt/hooks/dbt_google_hook.py @@ -3,12 +3,11 @@ import tarfile from tempfile import NamedTemporaryFile from typing import Any, Dict, List -from uuid import uuid4 from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.cloud_build import CloudBuildHook from airflow.providers.google.cloud.hooks.gcs import ( - GCSHook, _parse_gcs_url, gcs_object_is_directory, + GCSHook, _parse_gcs_url, ) from hooks.dbt_hook import DbtBaseHook @@ -131,7 +130,7 @@ def run_dbt(self, dbt_cmd: List[str]): } }, project_id=self.project_id, - wait=True, + wait=self.wait, timeout=self.timeout, metadata=self.env, ) From 5ac7f2ac71b8e23c4036777e8d52374a4a96664c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 18 Oct 2021 15:57:01 +0200 Subject: [PATCH 07/79] Fix the upload sources test --- tests/operators/test_dbt_operator.py | 33 +++++++++++----------------- 1 file changed, 13 insertions(+), 20 deletions(-) diff --git a/tests/operators/test_dbt_operator.py b/tests/operators/test_dbt_operator.py index bb4dc31..27544c8 100644 --- a/tests/operators/test_dbt_operator.py +++ b/tests/operators/test_dbt_operator.py @@ -76,35 +76,28 @@ def setUp(self): } self.dag = DAG('test_dag_id', default_args=args) + @patch('airflow_dbt.hooks.dbt_google_hook.NamedTemporaryFile') @patch('airflow_dbt.hooks.dbt_google_hook.CloudBuildHook') @patch('airflow_dbt.hooks.dbt_google_hook.GCSHook') - def test_dbt_deps(self, MockLocalCloudBuildHook, MockGCSHook): + def test_upload_files(self, MockGCSHook, MockCBHook, MockTempFile): + # Change the context provider returned name for the file + MockTempFile.return_value.__enter__.return_value.name = 'tempfile' operator = DbtRunOperator( task_id='test_dbt_run_on_cloud_build', dbt_hook=DbtCloudBuildHook( project_id='my-project-id', gcp_conn_id='my_conn_id', - gcs_staging_location='gs://my-bucket/certain-folder/' + dir='.', + gcs_staging_location='gs://my-bucket/certain-folder' + '/stored_dbt_files.tar.gz' ), dag=self.dag ) operator.execute(None) - MockLocalCloudBuildHook.assert_called_once_with(gcp_conn_id='my_conn_id') + MockCBHook.assert_called_once_with(gcp_conn_id='my_conn_id') MockGCSHook.assert_called_once_with(gcp_conn_id='my_conn_id') - MockGCSHook().upload.assert_called_once() - - -{ - 'steps': [{ - 'name': 'fishtownanalytics/dbt:0.21.0', 'entrypoint': '/bin/sh', - 'args': ['-c', 'dbt', 'run'], 'env': [] - }], - 'source': { - 'storageSource': { - 'bucket': 'my-bucket', - 'object': - 'certain-folder/dbt_staging_c5013d9965b54386bcf84ab76e42c848' - '.tar.gz' - } - } -} + MockGCSHook().upload.assert_called_once_with( + bucket_name='my-bucket', + object_name='certain-folder/stored_dbt_files.tar.gz', + filename='tempfile' + ) From 53243c01389ae0637725c8480b916f58070b5e5c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 18 Oct 2021 16:09:57 +0200 Subject: [PATCH 08/79] Add docstrings to all DbtBaseOperator subclases --- airflow_dbt/operators/dbt_operator.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index f4f2eaa..90930d5 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -1,4 +1,4 @@ -from typing import Any +from typing import Any, Dict from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults @@ -108,6 +108,7 @@ def execute(self, context: Any): class DbtRunOperator(DbtBaseOperator): + """Runs a dbt run command""" @apply_defaults def __init__(self, profiles_dir=None, target=None, *args, **kwargs): super().__init__( @@ -120,6 +121,7 @@ def __init__(self, profiles_dir=None, target=None, *args, **kwargs): class DbtTestOperator(DbtBaseOperator): + """Runs a dbt test command""" @apply_defaults def __init__(self, profiles_dir=None, target=None, *args, **kwargs): super().__init__( @@ -132,6 +134,7 @@ def __init__(self, profiles_dir=None, target=None, *args, **kwargs): class DbtDocsGenerateOperator(DbtBaseOperator): + """Runs a dbt docs generate command""" @apply_defaults def __init__(self, profiles_dir=None, target=None, *args, **kwargs): super().__init__( @@ -144,6 +147,7 @@ def __init__(self, profiles_dir=None, target=None, *args, **kwargs): class DbtSnapshotOperator(DbtBaseOperator): + """Runs a dbt snapshot command""" @apply_defaults def __init__(self, profiles_dir=None, target=None, *args, **kwargs): super().__init__( @@ -156,6 +160,7 @@ def __init__(self, profiles_dir=None, target=None, *args, **kwargs): class DbtSeedOperator(DbtBaseOperator): + """Runs a dbt seed command""" @apply_defaults def __init__(self, profiles_dir=None, target=None, *args, **kwargs): super().__init__( @@ -168,6 +173,7 @@ def __init__(self, profiles_dir=None, target=None, *args, **kwargs): class DbtDepsOperator(DbtBaseOperator): + """Runs a dbt deps command""" @apply_defaults def __init__(self, profiles_dir=None, target=None, *args, **kwargs): super().__init__( From 644f59470bfe6948033ddfb0a46701da5bec5c56 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 18 Oct 2021 16:10:24 +0200 Subject: [PATCH 09/79] Add `env` param to DbtBaseOperator to pass env variables to runtime --- airflow_dbt/operators/dbt_operator.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 90930d5..f45fc9d 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -16,6 +16,8 @@ class DbtBaseOperator(BaseOperator): :param target: If set, passed as the `--target` argument to the `dbt` command :type dir: str :param dir: The directory to run the CLI in + :param env: If set, passed to the dbt executor + :type env: dict :type vars: str :param vars: If set, passed as the `--vars` argument to the `dbt` command :type vars: dict @@ -53,7 +55,8 @@ class DbtBaseOperator(BaseOperator): def __init__(self, profiles_dir=None, target=None, - dir='.', + dir: str = '.', + env: Dict = None, vars=None, models=None, exclude=None, @@ -73,6 +76,7 @@ def __init__(self, self.profiles_dir = profiles_dir self.target = target self.dir = dir + self.env = {} if env is None else env self.vars = vars self.models = models self.full_refresh = full_refresh @@ -86,6 +90,7 @@ def __init__(self, self.base_command = base_command self.hook = dbt_hook if dbt_hook is not None else DbtCliHook( dir=dir, + env=self.env, dbt_bin=dbt_bin ) From 5cc650d4e84e03e24ac5ce569d9b37e42fc0d98f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 22 Oct 2021 12:06:18 +0200 Subject: [PATCH 10/79] Export the DbtCloudBuildHook from the hooks package --- airflow_dbt/hooks/__init__.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow_dbt/hooks/__init__.py b/airflow_dbt/hooks/__init__.py index 8644e88..a5c997a 100644 --- a/airflow_dbt/hooks/__init__.py +++ b/airflow_dbt/hooks/__init__.py @@ -1 +1,2 @@ from .dbt_hook import DbtCliHook +from .dbt_google_hook import DbtCloudBuildHook From db5e12132adbab1fc1ea90b429325042f4dc7f01 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 22 Oct 2021 12:07:01 +0200 Subject: [PATCH 11/79] Check that google provider API version is the right for the implementation (5) --- airflow_dbt/hooks/dbt_google_hook.py | 38 ++++++++++++++++++++-------- 1 file changed, 28 insertions(+), 10 deletions(-) diff --git a/airflow_dbt/hooks/dbt_google_hook.py b/airflow_dbt/hooks/dbt_google_hook.py index 6abbe11..dc7e47a 100644 --- a/airflow_dbt/hooks/dbt_google_hook.py +++ b/airflow_dbt/hooks/dbt_google_hook.py @@ -9,8 +9,24 @@ from airflow.providers.google.cloud.hooks.gcs import ( GCSHook, _parse_gcs_url, ) - -from hooks.dbt_hook import DbtBaseHook +from airflow.providers.google.get_provider_info import get_provider_info +from packaging import version + +from .dbt_hook import DbtBaseHook + +# Check we're using the right google provider version. As composer is the +# most brad used Airflow installation we will default to the latest version +# composer is using +google_providers_version = get_provider_info().get('versions')[0] +v_min = version.parse('5.0.0') +v_max = version.parse('6.0.0') +v_provider = version.parse(google_providers_version) +if not v_min <= v_provider < v_max: + raise Exception( + f'The provider "apache-airflow-providers-google" version "' + f'{google_providers_version}" is not compatible with the current API. ' + f'Please install a compatible version in the range [{v_min}, {v_max})"' + ) class DbtCloudBuildHook(DbtBaseHook): @@ -59,8 +75,6 @@ def __init__( env: Dict = None, dbt_bin='dbt', ): - logging.info(f'Files in "{dir}" will be uploaded to GCS with the ' - f'prefix "{gcs_staging_location}"') staging_bucket, staging_blob = _parse_gcs_url(gcs_staging_location) # we have provided something similar to # 'gs:///' @@ -87,10 +101,14 @@ def get_conn(self) -> Any: def upload_dbt_sources(self) -> None: """Upload sources from local to a staging location""" + logging.info( + f'Files in "{dir}" will be uploaded to GCS with the ' + f'prefix "gs://{self.gcs_staging_bucket}/{self.gcs_staging_blob}"' + ) gcs_hook = GCSHook(gcp_conn_id=self.gcp_conn_id) with \ - NamedTemporaryFile() as compressed_file, \ - tarfile.open(compressed_file.name, "w:gz") as tar: + NamedTemporaryFile() as compressed_file, \ + tarfile.open(compressed_file.name, "w:gz") as tar: tar.add(self.dir, arcname=os.path.basename(self.dir)) gcs_hook.upload( bucket_name=self.gcs_staging_bucket, @@ -115,7 +133,7 @@ def run_dbt(self, dbt_cmd: List[str]): self.upload_dbt_sources() results = self.cloud_build_hook.create_build( - build={ + body={ 'steps': [{ 'name': f'fishtownanalytics/dbt:{self.dbt_version}', 'entrypoint': '/bin/sh', @@ -130,9 +148,9 @@ def run_dbt(self, dbt_cmd: List[str]): } }, project_id=self.project_id, - wait=self.wait, - timeout=self.timeout, - metadata=self.env, + # wait=self.wait, + # timeout=self.timeout, + # metadata=self.env, ) logging.info( f'Triggered build {results["id"]}. You can find the logs at ' From f55624ae32500ac87206a8767427a488e28d9467 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 22 Oct 2021 12:12:46 +0200 Subject: [PATCH 12/79] Add project_dir flag to the command line The Airflow bash executor will run the command isolated in the worker. To be able to locate the profiles and code we have to tell dbt with the new implemented flag `dbt run --project-dir /path/to/my/project_dir --profiles-dir /path/to/profile.yaml` ` --- airflow_dbt/hooks/dbt_hook.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/airflow_dbt/hooks/dbt_hook.py b/airflow_dbt/hooks/dbt_hook.py index bf2c3fc..8897df0 100644 --- a/airflow_dbt/hooks/dbt_hook.py +++ b/airflow_dbt/hooks/dbt_hook.py @@ -31,6 +31,7 @@ def generate_dbt_cli_command( self, base_command: str, profiles_dir: str = None, + project_dir: str = None, target: str = None, vars: Dict[str, str] = None, full_refresh: bool = False, @@ -78,6 +79,9 @@ def generate_dbt_cli_command( if profiles_dir is not None: dbt_cmd.extend(['--profiles-dir', profiles_dir]) + if project_dir is not None: + dbt_cmd.extend(['--project-dir', project_dir]) + if target is not None: dbt_cmd.extend(['--target', target]) From ef26f88067901b14ee559bb9d108983d4886e7ce Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 22 Oct 2021 12:58:50 +0200 Subject: [PATCH 13/79] Remove cloud build timeout and wait. Thats google provider api v6, not 5 --- airflow_dbt/hooks/dbt_google_hook.py | 47 +++++++++++++++------------- 1 file changed, 25 insertions(+), 22 deletions(-) diff --git a/airflow_dbt/hooks/dbt_google_hook.py b/airflow_dbt/hooks/dbt_google_hook.py index dc7e47a..36aaebd 100644 --- a/airflow_dbt/hooks/dbt_google_hook.py +++ b/airflow_dbt/hooks/dbt_google_hook.py @@ -1,5 +1,6 @@ import logging import os +import pprint import tarfile from tempfile import NamedTemporaryFile from typing import Any, Dict, List @@ -68,12 +69,11 @@ def __init__( project_id: str, dir: str = None, gcs_staging_location: str = None, - timeout: str = None, - wait: bool = True, gcp_conn_id: str = "google_cloud_default", dbt_version: str = '0.21.0', env: Dict = None, - dbt_bin='dbt', + dbt_bin='', + service_account=None, ): staging_bucket, staging_blob = _parse_gcs_url(gcs_staging_location) # we have provided something similar to @@ -90,8 +90,7 @@ def __init__( self.cloud_build_hook = CloudBuildHook(gcp_conn_id=gcp_conn_id) self.gcp_conn_id = gcp_conn_id self.project_id = project_id - self.timeout = timeout - self.wait = wait + self.service_account = service_account super().__init__(dir=dir, env=env, dbt_bin=dbt_bin) @@ -132,25 +131,29 @@ def run_dbt(self, dbt_cmd: List[str]): if self.dir is not None: self.upload_dbt_sources() - results = self.cloud_build_hook.create_build( - body={ - 'steps': [{ - 'name': f'fishtownanalytics/dbt:{self.dbt_version}', - 'entrypoint': '/bin/sh', - 'args': ['-c', *dbt_cmd], - 'env': [f'{k}={v}' for k, v in self.env.items()] - }], - 'source': { - 'storageSource': { - "bucket": self.gcs_staging_bucket, - "object": self.gcs_staging_blob, - } + cloud_build_config = { + 'steps': [{ + 'name': f'fishtownanalytics/dbt:{self.dbt_version}', + 'args': dbt_cmd, + 'env': [f'{k}={v}' for k, v in self.env.items()] + }], + 'source': { + 'storageSource': { + "bucket": self.gcs_staging_bucket, + "object": self.gcs_staging_blob, } - }, + } + } + + if self.service_account is not None: + cloud_build_config['serviceAccount'] = self.service_account + + cloud_build_config_str = pprint.pformat(cloud_build_config) + logging.info(f'Running the following cloud build config:\n{cloud_build_config_str}') + + results = self.cloud_build_hook.create_build( + body=cloud_build_config, project_id=self.project_id, - # wait=self.wait, - # timeout=self.timeout, - # metadata=self.env, ) logging.info( f'Triggered build {results["id"]}. You can find the logs at ' From 129aa933071ecec9fe9b91faddc8b780e65286e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 22 Oct 2021 12:59:50 +0200 Subject: [PATCH 14/79] If dbt_bin is provided None or empty just don't add it to the command That comes useful when running the command inside cloud run, since the entrypoint for the command is already `dbt`, and this `dbt` is already in the PATH of the image --- airflow_dbt/hooks/dbt_hook.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/airflow_dbt/hooks/dbt_hook.py b/airflow_dbt/hooks/dbt_hook.py index 8897df0..67cbc75 100644 --- a/airflow_dbt/hooks/dbt_hook.py +++ b/airflow_dbt/hooks/dbt_hook.py @@ -74,7 +74,15 @@ def generate_dbt_cli_command( command :type select: str """ - dbt_cmd = [self.dbt_bin, base_command] + # if there's no bin do not append it. Rather generate the command + # without the `/path/to/dbt` prefix. That is useful for running it + # inside containers + if self.dbt_bin == '' or self.dbt_bin is None: + dbt_cmd = [] + else: + dbt_cmd = [self.dbt_bin] + + dbt_cmd.append(base_command) if profiles_dir is not None: dbt_cmd.extend(['--profiles-dir', profiles_dir]) From 289d563bf5783a286b0903373ba1370007539608 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 22 Oct 2021 13:02:39 +0200 Subject: [PATCH 15/79] Add `project_dir` option flag Local execution might not take place in the same folder. It depends on the underlying hook implementation. This way we ensure we're pointing to the folder with an absolute path even if run outside the project folder --- airflow_dbt/operators/dbt_operator.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index f45fc9d..52aff60 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -1,3 +1,4 @@ +import logging from typing import Any, Dict from airflow.models import BaseOperator @@ -54,8 +55,9 @@ class DbtBaseOperator(BaseOperator): @apply_defaults def __init__(self, profiles_dir=None, - target=None, + project_dir = None, dir: str = '.', + target=None, env: Dict = None, vars=None, models=None, @@ -74,8 +76,8 @@ def __init__(self, super(DbtBaseOperator, self).__init__(*args, **kwargs) self.profiles_dir = profiles_dir + self.project_dir = project_dir if project_dir is not None else dir self.target = target - self.dir = dir self.env = {} if env is None else env self.vars = vars self.models = models @@ -99,6 +101,7 @@ def execute(self, context: Any): dbt_cli_command = self.hook.generate_dbt_cli_command( base_command=self.base_command, profiles_dir=self.profiles_dir, + project_dir=self.project_dir, target=self.target, vars=self.vars, full_refresh=self.full_refresh, @@ -109,6 +112,7 @@ def execute(self, context: Any): select=self.select, warn_error=self.warn_error, ) + logging.info(f'Running dbt command "{dbt_cli_command}"') self.hook.run_dbt(dbt_cli_command) From 081222da4e57cd28e49e6dde79f543204fb47e36 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Sat, 23 Oct 2021 00:05:11 +0200 Subject: [PATCH 16/79] Add the `use_color` flag (#43). If set adds the dbt flag Also make the command build function more idiomatic by using `append` to append single commands (flags) and `extend` when we want to append several. --- airflow_dbt/hooks/dbt_hook.py | 23 +++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/airflow_dbt/hooks/dbt_hook.py b/airflow_dbt/hooks/dbt_hook.py index 67cbc75..3bc1cdd 100644 --- a/airflow_dbt/hooks/dbt_hook.py +++ b/airflow_dbt/hooks/dbt_hook.py @@ -21,25 +21,25 @@ class DbtBaseHook(BaseHook, ABC): :type dbt_bin: str """ - def __init__(self, dir: str = '.', env: Dict = None, dbt_bin='dbt'): + def __init__(self, env: Dict = None, dbt_bin='dbt'): super().__init__() - self.dir = dir self.env = env if env is not None else {} self.dbt_bin = dbt_bin def generate_dbt_cli_command( self, base_command: str, - profiles_dir: str = None, - project_dir: str = None, + profiles_dir: str = '.', + project_dir: str = '.', target: str = None, - vars: Dict[str, str] = None, + vars: Dict = None, full_refresh: bool = False, data: bool = False, schema: bool = False, models: str = None, exclude: str = None, select: str = None, + use_colors: bool = None, warn_error: bool = False, ) -> List[str]: """ @@ -97,26 +97,30 @@ def generate_dbt_cli_command( dbt_cmd.extend(['--vars', json.dumps(vars)]) if data: - dbt_cmd.extend(['--data']) + dbt_cmd.append('--data') if schema: - dbt_cmd.extend(['--schema']) + dbt_cmd.append('--schema') if models is not None: dbt_cmd.extend(['--models', models]) if exclude is not None: - dbt_cmd.extend(['--exclude', self]) + dbt_cmd.extend(['--exclude', exclude]) if select is not None: dbt_cmd.extend(['--select', select]) if full_refresh: - dbt_cmd.extend(['--full-refresh']) + dbt_cmd.append('--full-refresh') if warn_error: dbt_cmd.insert(1, '--warn-error') + if use_colors is not None: + colors_flag = "--use-colors" if use_colors else "--no-use-colors" + dbt_cmd.append(colors_flag) + return dbt_cmd @abstractmethod @@ -161,7 +165,6 @@ def run_dbt(self, dbt_cmd: Union[str, List[str]]): self.sp.run_command( command=dbt_cmd, env=self.env, - cwd=self.dir, ) def on_kill(self): From bff2d5c33e0b08dc8c85ae51e24209159afb80b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Sat, 23 Oct 2021 00:32:48 +0200 Subject: [PATCH 17/79] Template all the fields from the DbtBaseOperator (#46 and #30) --- airflow_dbt/operators/dbt_operator.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 52aff60..65a5efb 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -50,7 +50,9 @@ class DbtBaseOperator(BaseOperator): ui_color = '#d6522a' - template_fields = ['vars'] + template_fields = ['profiles_dir', 'project_dir', 'target', 'env', + 'vars', 'models', 'exclude', 'select', 'dbt_bin', 'verbose', + 'warn_error', 'full_refresh', 'data', 'schema', 'base_command'] @apply_defaults def __init__(self, From 6a459e8d5a5bba45439404034844149bb7c5c823 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 25 Oct 2021 11:41:37 +0200 Subject: [PATCH 18/79] Refactor each hook into it's own file --- airflow_dbt/hooks/__init__.py | 5 +- airflow_dbt/hooks/{dbt_hook.py => base.py} | 59 +++---------------- airflow_dbt/hooks/cli.py | 51 ++++++++++++++++ .../hooks/{dbt_google_hook.py => google.py} | 8 ++- airflow_dbt/operators/dbt_operator.py | 2 +- 5 files changed, 70 insertions(+), 55 deletions(-) rename airflow_dbt/hooks/{dbt_hook.py => base.py} (70%) create mode 100644 airflow_dbt/hooks/cli.py rename airflow_dbt/hooks/{dbt_google_hook.py => google.py} (97%) diff --git a/airflow_dbt/hooks/__init__.py b/airflow_dbt/hooks/__init__.py index a5c997a..b08db3d 100644 --- a/airflow_dbt/hooks/__init__.py +++ b/airflow_dbt/hooks/__init__.py @@ -1,2 +1,3 @@ -from .dbt_hook import DbtCliHook -from .dbt_google_hook import DbtCloudBuildHook +from .base import DbtBaseHook +from .cli import DbtCliHook +from .google import DbtCloudBuildHook diff --git a/airflow_dbt/hooks/dbt_hook.py b/airflow_dbt/hooks/base.py similarity index 70% rename from airflow_dbt/hooks/dbt_hook.py rename to airflow_dbt/hooks/base.py index 3bc1cdd..9f42622 100644 --- a/airflow_dbt/hooks/dbt_hook.py +++ b/airflow_dbt/hooks/base.py @@ -2,18 +2,15 @@ import json from abc import ABC, abstractmethod -from typing import Any, Dict, List, Union +from typing import Dict, List, Union from airflow.hooks.base_hook import BaseHook -from airflow.hooks.subprocess import SubprocessHook class DbtBaseHook(BaseHook, ABC): """ Simple wrapper around the dbt CLI. - :type dir: str - :param dir: The directory to run the CLI in :type env: dict :param env: If set, passed to the dbt executor :param dbt_bin: The `dbt` CLI. Defaults to `dbt`, so assumes it's on your @@ -51,6 +48,10 @@ def generate_dbt_cli_command( :param profiles_dir: If set, passed as the `--profiles-dir` argument to the `dbt` command :type profiles_dir: str + :param project_dir: If set, passed as the `--project-dir` argument to + the `dbt` command. It is required but by default points to the + current folder: '.' + :type project_dir: str :param target: If set, passed as the `--target` argument to the `dbt` command :type vars: Union[str, dict] @@ -68,10 +69,12 @@ def generate_dbt_cli_command( :param warn_error: If `True`, treat warnings as errors. :type warn_error: bool :param exclude: If set, passed as the `--exclude` argument to the `dbt` - command + command :type exclude: str + :param use_colors: If set it adds the flag `--use-colors` or + `--no-use-colors`, depending if True or False. :param select: If set, passed as the `--select` argument to the `dbt` - command + command :type select: str """ # if there's no bin do not append it. Rather generate the command @@ -126,47 +129,3 @@ def generate_dbt_cli_command( @abstractmethod def run_dbt(self, dbt_cmd: Union[str, List[str]]): """Run the dbt command""" - - -class DbtCliHook(DbtBaseHook): - """ - Run the dbt command in the same airflow worker the task is being run. - This requires the `dbt` python package to be installed in it first. Also - the dbt_bin path might not be set in the `PATH` variable, so it could be - necessary to set it in the constructor. - - :type dir: str - :param dir: The directory to run the CLI in - :type env: dict - :param env: If set, passed to the dbt executor - :param dbt_bin: The `dbt` CLI. Defaults to `dbt`, so assumes it's on your - `PATH` - :type dbt_bin: str - """ - - def __init__(self, dir: str = '.', env: Dict = None, dbt_bin='dbt'): - self.sp = SubprocessHook() - super().__init__(dir=dir, env=env, dbt_bin=dbt_bin) - - def get_conn(self) -> Any: - """ - Return the subprocess connection, which isn't implemented, just for - conformity - """ - return self.sp.get_conn() - - def run_dbt(self, dbt_cmd: Union[str, List[str]]): - """ - Run the dbt cli - - :param dbt_cmd: The dbt whole command to run - :type dbt_cmd: List[str] - """ - self.sp.run_command( - command=dbt_cmd, - env=self.env, - ) - - def on_kill(self): - """Kill the open subprocess if the task gets killed by Airflow""" - self.sp.send_sigterm() diff --git a/airflow_dbt/hooks/cli.py b/airflow_dbt/hooks/cli.py new file mode 100644 index 0000000..9a8c798 --- /dev/null +++ b/airflow_dbt/hooks/cli.py @@ -0,0 +1,51 @@ +from __future__ import print_function + +from typing import Any, Dict, List, Union + +from airflow.hooks.subprocess import SubprocessHook + +from hooks.base import DbtBaseHook + + +class DbtCliHook(DbtBaseHook): + """ + Run the dbt command in the same airflow worker the task is being run. + This requires the `dbt` python package to be installed in it first. Also + the dbt_bin path might not be set in the `PATH` variable, so it could be + necessary to set it in the constructor. + + :type dir: str + :param dir: The directory to run the CLI in + :type env: dict + :param env: If set, passed to the dbt executor + :param dbt_bin: The `dbt` CLI. Defaults to `dbt`, so assumes it's on your + `PATH` + :type dbt_bin: str + """ + + def __init__(self, env: Dict = None, dbt_bin='dbt'): + self.sp = SubprocessHook() + super().__init__(env=env, dbt_bin=dbt_bin) + + def get_conn(self) -> Any: + """ + Return the subprocess connection, which isn't implemented, just for + conformity + """ + return self.sp.get_conn() + + def run_dbt(self, dbt_cmd: Union[str, List[str]]): + """ + Run the dbt cli + + :param dbt_cmd: The dbt whole command to run + :type dbt_cmd: List[str] + """ + self.sp.run_command( + command=dbt_cmd, + env=self.env, + ) + + def on_kill(self): + """Kill the open subprocess if the task gets killed by Airflow""" + self.sp.send_sigterm() diff --git a/airflow_dbt/hooks/dbt_google_hook.py b/airflow_dbt/hooks/google.py similarity index 97% rename from airflow_dbt/hooks/dbt_google_hook.py rename to airflow_dbt/hooks/google.py index 36aaebd..4912719 100644 --- a/airflow_dbt/hooks/dbt_google_hook.py +++ b/airflow_dbt/hooks/google.py @@ -67,7 +67,7 @@ class DbtCloudBuildHook(DbtBaseHook): def __init__( self, project_id: str, - dir: str = None, + # dir: str = None, gcs_staging_location: str = None, gcp_conn_id: str = "google_cloud_default", dbt_version: str = '0.21.0', @@ -92,7 +92,11 @@ def __init__( self.project_id = project_id self.service_account = service_account - super().__init__(dir=dir, env=env, dbt_bin=dbt_bin) + super().__init__( + # dir=dir, + env=env, + dbt_bin=dbt_bin + ) def get_conn(self) -> Any: """Returns the cloud build connection, which is a gcp connection""" diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 65a5efb..2761d8f 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -93,7 +93,7 @@ def __init__(self, self.warn_error = warn_error self.base_command = base_command self.hook = dbt_hook if dbt_hook is not None else DbtCliHook( - dir=dir, + # dir=dir, env=self.env, dbt_bin=dbt_bin ) From dcaa3bb087b49b08a90a8a4b8b1204f10cb5c1f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 25 Oct 2021 14:12:32 +0200 Subject: [PATCH 19/79] Remove upload functionality, provided by `LocalFilesystemToGCSOperator` --- airflow_dbt/hooks/google.py | 27 +-------------------------- 1 file changed, 1 insertion(+), 26 deletions(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index 4912719..ba49470 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -72,7 +72,7 @@ def __init__( gcp_conn_id: str = "google_cloud_default", dbt_version: str = '0.21.0', env: Dict = None, - dbt_bin='', + dbt_bin=None, service_account=None, ): staging_bucket, staging_blob = _parse_gcs_url(gcs_staging_location) @@ -93,7 +93,6 @@ def __init__( self.service_account = service_account super().__init__( - # dir=dir, env=env, dbt_bin=dbt_bin ) @@ -102,23 +101,6 @@ def get_conn(self) -> Any: """Returns the cloud build connection, which is a gcp connection""" return self.cloud_build_hook.get_conn() - def upload_dbt_sources(self) -> None: - """Upload sources from local to a staging location""" - logging.info( - f'Files in "{dir}" will be uploaded to GCS with the ' - f'prefix "gs://{self.gcs_staging_bucket}/{self.gcs_staging_blob}"' - ) - gcs_hook = GCSHook(gcp_conn_id=self.gcp_conn_id) - with \ - NamedTemporaryFile() as compressed_file, \ - tarfile.open(compressed_file.name, "w:gz") as tar: - tar.add(self.dir, arcname=os.path.basename(self.dir)) - gcs_hook.upload( - bucket_name=self.gcs_staging_bucket, - object_name=self.gcs_staging_blob, - filename=compressed_file.name, - ) - def run_dbt(self, dbt_cmd: List[str]): """ Run the dbt cli @@ -128,13 +110,6 @@ def run_dbt(self, dbt_cmd: List[str]): """ """See: https://cloud.google.com/cloud-build/docs/api/reference/rest /v1/projects.builds""" - - # if we indicate that the sources are in a local directory by setting - # the "dir" pointing to a local path, then those sources will be - # uploaded to the expected blob - if self.dir is not None: - self.upload_dbt_sources() - cloud_build_config = { 'steps': [{ 'name': f'fishtownanalytics/dbt:{self.dbt_version}', From f23490052fe24efa794e2f59bf3e01303aa709a0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 25 Oct 2021 14:14:13 +0200 Subject: [PATCH 20/79] Adjust imports after refactor --- airflow_dbt/hooks/base.py | 4 +-- airflow_dbt/hooks/cli.py | 2 +- airflow_dbt/hooks/google.py | 2 +- airflow_dbt/operators/dbt_operator.py | 43 ++++++++++++++------------- 4 files changed, 26 insertions(+), 25 deletions(-) diff --git a/airflow_dbt/hooks/base.py b/airflow_dbt/hooks/base.py index 9f42622..9871e8e 100644 --- a/airflow_dbt/hooks/base.py +++ b/airflow_dbt/hooks/base.py @@ -26,8 +26,8 @@ def __init__(self, env: Dict = None, dbt_bin='dbt'): def generate_dbt_cli_command( self, base_command: str, - profiles_dir: str = '.', - project_dir: str = '.', + profiles_dir: str = None, + project_dir: str = None, target: str = None, vars: Dict = None, full_refresh: bool = False, diff --git a/airflow_dbt/hooks/cli.py b/airflow_dbt/hooks/cli.py index 9a8c798..c177482 100644 --- a/airflow_dbt/hooks/cli.py +++ b/airflow_dbt/hooks/cli.py @@ -4,7 +4,7 @@ from airflow.hooks.subprocess import SubprocessHook -from hooks.base import DbtBaseHook +from .base import DbtBaseHook class DbtCliHook(DbtBaseHook): diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index ba49470..4722319 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -13,7 +13,7 @@ from airflow.providers.google.get_provider_info import get_provider_info from packaging import version -from .dbt_hook import DbtBaseHook +from .base import DbtBaseHook # Check we're using the right google provider version. As composer is the # most brad used Airflow installation we will default to the latest version diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 2761d8f..8ab76ef 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -4,7 +4,7 @@ from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow_dbt.hooks.dbt_hook import DbtCliHook +from airflow_dbt.hooks import DbtCliHook class DbtBaseOperator(BaseOperator): @@ -55,26 +55,27 @@ class DbtBaseOperator(BaseOperator): 'warn_error', 'full_refresh', 'data', 'schema', 'base_command'] @apply_defaults - def __init__(self, - profiles_dir=None, - project_dir = None, - dir: str = '.', - target=None, - env: Dict = None, - vars=None, - models=None, - exclude=None, - select=None, - dbt_bin='dbt', - verbose=True, - warn_error=False, - full_refresh=False, - data=False, - schema=False, - dbt_hook=None, - base_command=None, - *args, - **kwargs): + def __init__( + self, + profiles_dir=None, + project_dir=None, + dir: str = None, + target=None, + env: Dict = None, + vars=None, + models=None, + exclude=None, + select=None, + dbt_bin='dbt', + verbose=True, + warn_error=False, + full_refresh=False, + data=False, + schema=False, + dbt_hook=None, + base_command=None, + *args, + **kwargs): super(DbtBaseOperator, self).__init__(*args, **kwargs) self.profiles_dir = profiles_dir From c58289919f274be55cd4b3b82a420fc525c4f036 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 25 Oct 2021 14:15:17 +0200 Subject: [PATCH 21/79] Test Cloud Build DBT hook and remove test cloudbuild upload Also fix imports and fix a dbt cli test --- tests/__init__.py | 0 tests/hooks/test_dbt_hook.py | 43 +++++++++++++++++++++++++--- tests/operators/test_dbt_operator.py | 30 +------------------ 3 files changed, 40 insertions(+), 33 deletions(-) delete mode 100644 tests/__init__.py diff --git a/tests/__init__.py b/tests/__init__.py deleted file mode 100644 index e69de29..0000000 diff --git a/tests/hooks/test_dbt_hook.py b/tests/hooks/test_dbt_hook.py index e1990b8..88cefbe 100644 --- a/tests/hooks/test_dbt_hook.py +++ b/tests/hooks/test_dbt_hook.py @@ -1,19 +1,20 @@ from unittest import TestCase, mock +from unittest.mock import patch from airflow.hooks.subprocess import SubprocessHook -from airflow_dbt.hooks.dbt_hook import DbtCliHook +from airflow_dbt.hooks.cli import DbtCliHook +from airflow_dbt.hooks.google import DbtCloudBuildHook -class TestDbtHook(TestCase): +class TestDbtCliHook(TestCase): @mock.patch.object(SubprocessHook, 'run_command') def test_sub_commands(self, mock_run_command): hook = DbtCliHook() hook.run_dbt(['dbt', 'docs', 'generate']) mock_run_command.assert_called_once_with( command=['dbt', 'docs', 'generate'], - env={}, - cwd='.', + env={} ) def test_vars(self): @@ -25,3 +26,37 @@ def test_vars(self): assert generated_command == ['dbt', 'run', '--vars', '{"foo": "bar", "baz": "true"}'] + + +class TestDbtCloudBuildHook(TestCase): + @patch('airflow_dbt.hooks.google.CloudBuildHook') + def test_create_build(self, MockCloudBuildHook): + mock_create_build = MockCloudBuildHook().create_build + hook = DbtCloudBuildHook( + project_id='test_project_id', + gcs_staging_location='gs://hello/file.tar.gz', + dbt_version='0.10.10', + env={'TEST_ENV_VAR': 'test'}, + service_account='robot@mail.com' + ) + hook.run_dbt(['docs', 'generate']) + + expected_body = { + 'steps': [{ + 'name': f'fishtownanalytics/dbt:0.10.10', + 'args': ['docs', 'generate'], + 'env': ['TEST_ENV_VAR=test'] + }], + 'source': { + 'storageSource': { + 'bucket': 'hello', + 'object': 'file.tar.gz', + } + }, + 'serviceAccount': 'robot@mail.com' + } + + mock_create_build.assert_called_once_with( + body=expected_body, + project_id='test_project_id' + ) \ No newline at end of file diff --git a/tests/operators/test_dbt_operator.py b/tests/operators/test_dbt_operator.py index 27544c8..48cd1ba 100644 --- a/tests/operators/test_dbt_operator.py +++ b/tests/operators/test_dbt_operator.py @@ -1,11 +1,9 @@ import datetime from unittest import TestCase, mock -from unittest.mock import patch from airflow import DAG, configuration -from airflow_dbt.hooks.dbt_google_hook import DbtCloudBuildHook -from airflow_dbt.hooks.dbt_hook import DbtCliHook +from airflow_dbt.hooks.cli import DbtCliHook from airflow_dbt.operators.dbt_operator import ( DbtDepsOperator, DbtRunOperator, DbtSeedOperator, DbtSnapshotOperator, DbtTestOperator, @@ -75,29 +73,3 @@ def setUp(self): 'start_date': datetime.datetime(2020, 2, 27) } self.dag = DAG('test_dag_id', default_args=args) - - @patch('airflow_dbt.hooks.dbt_google_hook.NamedTemporaryFile') - @patch('airflow_dbt.hooks.dbt_google_hook.CloudBuildHook') - @patch('airflow_dbt.hooks.dbt_google_hook.GCSHook') - def test_upload_files(self, MockGCSHook, MockCBHook, MockTempFile): - # Change the context provider returned name for the file - MockTempFile.return_value.__enter__.return_value.name = 'tempfile' - operator = DbtRunOperator( - task_id='test_dbt_run_on_cloud_build', - dbt_hook=DbtCloudBuildHook( - project_id='my-project-id', - gcp_conn_id='my_conn_id', - dir='.', - gcs_staging_location='gs://my-bucket/certain-folder' - '/stored_dbt_files.tar.gz' - ), - dag=self.dag - ) - operator.execute(None) - MockCBHook.assert_called_once_with(gcp_conn_id='my_conn_id') - MockGCSHook.assert_called_once_with(gcp_conn_id='my_conn_id') - MockGCSHook().upload.assert_called_once_with( - bucket_name='my-bucket', - object_name='certain-folder/stored_dbt_files.tar.gz', - filename='tempfile' - ) From efdb64d6cf60d53a0d931d8442dfeceae5a6f12c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 26 Oct 2021 01:37:20 +0200 Subject: [PATCH 22/79] Fix test params for generate_dbt_cli_command --- tests/hooks/test_dbt_hook.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/hooks/test_dbt_hook.py b/tests/hooks/test_dbt_hook.py index 88cefbe..199dd2d 100644 --- a/tests/hooks/test_dbt_hook.py +++ b/tests/hooks/test_dbt_hook.py @@ -20,7 +20,8 @@ def test_sub_commands(self, mock_run_command): def test_vars(self): hook = DbtCliHook() generated_command = hook.generate_dbt_cli_command( - 'run', + dbt_bin='dbt', + command='run', vars={"foo": "bar", "baz": "true"} ) From 678989c141220c7681e4fe17d595e553927bd031 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 26 Oct 2021 01:38:16 +0200 Subject: [PATCH 23/79] Export DbtBaseOperator and DbtCloudBuildOperator from the .operators pkg --- airflow_dbt/operators/__init__.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/airflow_dbt/operators/__init__.py b/airflow_dbt/operators/__init__.py index 295d53b..5b75e29 100644 --- a/airflow_dbt/operators/__init__.py +++ b/airflow_dbt/operators/__init__.py @@ -1,8 +1,10 @@ from .dbt_operator import ( + DbtBaseOperator, DbtSeedOperator, DbtSnapshotOperator, DbtRunOperator, DbtTestOperator, DbtDocsGenerateOperator, - DbtDepsOperator + DbtDepsOperator, + DbtCloudBuildOperator ) From 3ff8c4f281466998eb12f1638257bc1703c903a0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 26 Oct 2021 01:39:57 +0200 Subject: [PATCH 24/79] Add missing dbt flags, add optional config and Config type to enforce it --- airflow_dbt/dbt_command_config.py | 68 +++++++ airflow_dbt/hooks/base.py | 143 +++++++++++--- airflow_dbt/operators/dbt_operator.py | 270 ++++++++++++++++---------- 3 files changed, 356 insertions(+), 125 deletions(-) create mode 100644 airflow_dbt/dbt_command_config.py diff --git a/airflow_dbt/dbt_command_config.py b/airflow_dbt/dbt_command_config.py new file mode 100644 index 0000000..038e733 --- /dev/null +++ b/airflow_dbt/dbt_command_config.py @@ -0,0 +1,68 @@ +import sys + +if sys.version_info[0] == 3 and sys.version_info[1] >= 8: + from typing import TypedDict +else: + from typing_extensions import TypedDict + + +class DbtCommandConfig(TypedDict, total=False): + """ + Holds the structure of a dictionary containing dbt config. Provides the + types and names for each one, and also helps shortening the constructor + since we can nest it and reuse it + """ + # global flags + version: bool + record_timing_info: bool + debug: bool + log_format: str # either 'text', 'json' or 'default' + write_json: bool + strict: bool + warn_error: bool + partial_parse: bool + use_experimental_parser: bool + use_colors: bool + + # per command flags + profiles_dir: str + project_dir: str + target: str + vars: dict + models: str + exclude: str + verbose: bool + + # run specific + full_refresh: bool + profile: str + + # docs specific + no_compile: bool + + # debug specific + config_dir: str + + # ls specific + resource_type: str # models, snapshots, seeds, tests, and sources. + # '--resource-type' + select: str + models: str + exclude: str + selector: str + output: str + output_keys: str + + # rpc specific + host: str + port: int + + # run specific + fail_fast: bool + + # run-operation specific + args: dict + + # test specific + data: bool + schema: bool diff --git a/airflow_dbt/hooks/base.py b/airflow_dbt/hooks/base.py index 9871e8e..e9394d2 100644 --- a/airflow_dbt/hooks/base.py +++ b/airflow_dbt/hooks/base.py @@ -25,26 +25,52 @@ def __init__(self, env: Dict = None, dbt_bin='dbt'): def generate_dbt_cli_command( self, - base_command: str, + dbt_bin: str = None, + command: str = None, + # global flags + version: bool = False, + record_timing_info: bool = False, + debug: bool = False, + log_format: str = None, # either 'text', 'json' or 'default' + write_json: bool = None, + strict: bool = False, + warn_error: bool = False, + partial_parse: bool = False, + use_experimental_parser: bool = False, + use_colors: bool = None, + # command specific config profiles_dir: str = None, project_dir: str = None, + profile: str = None, target: str = None, + config_dir: str = None, + resource_type: str = None, vars: Dict = None, + # run specific full_refresh: bool = False, + # ls specific data: bool = False, schema: bool = False, models: str = None, exclude: str = None, select: str = None, - use_colors: bool = None, - warn_error: bool = False, + selector: str = None, + output: str = None, + output_keys: str = None, + # rpc specific + host: str = None, + port: str = None, + # test specific + fail_fast: bool = False, + args: dict = None, + no_compile: bool = False, ) -> List[str]: """ Generate the command that will be run based on class properties, presets and dbt commands - :param base_command: The dbt sub-command to run - :type base_command: str + :param command: The dbt sub-command to run + :type command: str :param profiles_dir: If set, passed as the `--profiles-dir` argument to the `dbt` command :type profiles_dir: str @@ -77,33 +103,75 @@ def generate_dbt_cli_command( command :type select: str """ + + dbt_cmd: List[str] = [] + # if there's no bin do not append it. Rather generate the command # without the `/path/to/dbt` prefix. That is useful for running it - # inside containers - if self.dbt_bin == '' or self.dbt_bin is None: - dbt_cmd = [] - else: - dbt_cmd = [self.dbt_bin] + # inside containers that have already set the entrypoint. + if dbt_bin is not None and not dbt_bin == '': + dbt_cmd.append(dbt_bin) + + # add global flags at the beginning + if version: + dbt_cmd.append('--version') - dbt_cmd.append(base_command) + if record_timing_info: + dbt_cmd.append('--record-timing-info') + + if debug: + dbt_cmd.append('--debug') + + if log_format is not None: + dbt_cmd.extend(['--log-format', log_format]) + + if write_json is not None: + write_json_flag = '--write-json' if write_json else \ + '--no-write-json' + dbt_cmd.append(write_json_flag) + + if strict: + dbt_cmd.append('--strict') + + if warn_error: + dbt_cmd.append('--warn-error') + if partial_parse: + dbt_cmd.append('--partial-parse') + + if use_experimental_parser: + dbt_cmd.append('--use-experimental-parser') + + if use_colors is not None: + colors_flag = "--use-colors" if use_colors else "--no-use-colors" + dbt_cmd.append(colors_flag) + + # appends the main command + dbt_cmd.append(command) + + # appends configuration relative to the command if profiles_dir is not None: dbt_cmd.extend(['--profiles-dir', profiles_dir]) if project_dir is not None: dbt_cmd.extend(['--project-dir', project_dir]) + if profile is not None: + dbt_cmd.extend(['--profile', profile]) + if target is not None: dbt_cmd.extend(['--target', target]) - if vars is not None: - dbt_cmd.extend(['--vars', json.dumps(vars)]) + # debug specific + if config_dir is not None: + dbt_cmd.extend(['--config-dir', config_dir]) - if data: - dbt_cmd.append('--data') + # ls specific + if resource_type is not None: + dbt_cmd.extend(['--resource-type', resource_type]) - if schema: - dbt_cmd.append('--schema') + if select is not None: + dbt_cmd.extend(['--select', select]) if models is not None: dbt_cmd.extend(['--models', models]) @@ -111,18 +179,45 @@ def generate_dbt_cli_command( if exclude is not None: dbt_cmd.extend(['--exclude', exclude]) - if select is not None: - dbt_cmd.extend(['--select', select]) + if selector is not None: + dbt_cmd.extend(['--selector', selector]) + if output is not None: + dbt_cmd.extend(['--output', output]) + + if output_keys is not None: + dbt_cmd.extend(['--output-keys', output_keys]) + + # rpc specific + if host is not None: + dbt_cmd.extend(['--host', host]) + + if port is not None: + dbt_cmd.extend(['--port', str(port)]) + + # run specific if full_refresh: dbt_cmd.append('--full-refresh') - if warn_error: - dbt_cmd.insert(1, '--warn-error') + if fail_fast: + dbt_cmd.append('--fail-fast') - if use_colors is not None: - colors_flag = "--use-colors" if use_colors else "--no-use-colors" - dbt_cmd.append(colors_flag) + if vars is not None: + dbt_cmd.extend(['--vars', json.dumps(vars)]) + + # run-operation specific + if args is not None: + dbt_cmd.extend(['--args', json.dumps(args)]) + + # test specific + if data: + dbt_cmd.append('--data') + + if schema: + dbt_cmd.append('--schema') + + if no_compile: + dbt_cmd.append('--no-compile') return dbt_cmd diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 8ab76ef..a4caf9b 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -1,10 +1,11 @@ import logging -from typing import Any, Dict +from typing import Any, Dict, Optional from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow_dbt.hooks import DbtCliHook +from airflow_dbt.hooks import DbtCliHook, DbtCloudBuildHook +from airflow_dbt.dbt_command_config import DbtCommandConfig class DbtBaseOperator(BaseOperator): @@ -12,9 +13,11 @@ class DbtBaseOperator(BaseOperator): Base dbt operator All other dbt operators are derived from this operator. - :param profiles_dir: If set, passed as the `--profiles-dir` argument to the `dbt` command + :param profiles_dir: If set, passed as the `--profiles-dir` argument to + the `dbt` command :type profiles_dir: str - :param target: If set, passed as the `--target` argument to the `dbt` command + :param target: If set, passed as the `--target` argument to the `dbt` + command :type dir: str :param dir: The directory to run the CLI in :param env: If set, passed to the dbt executor @@ -24,15 +27,19 @@ class DbtBaseOperator(BaseOperator): :type vars: dict :param full_refresh: If `True`, will fully-refresh incremental models. :type full_refresh: bool - :param models: If set, passed as the `--models` argument to the `dbt` command + :param models: If set, passed as the `--models` argument to the `dbt` + command :type models: str :param warn_error: If `True`, treat warnings as errors. :type warn_error: bool - :param exclude: If set, passed as the `--exclude` argument to the `dbt` command + :param exclude: If set, passed as the `--exclude` argument to the `dbt` + command :type exclude: str - :param select: If set, passed as the `--select` argument to the `dbt` command + :param select: If set, passed as the `--select` argument to the `dbt` + command :type select: str - :param dbt_bin: The `dbt` CLI. Defaults to `dbt`, so assumes it's on your `PATH` + :param dbt_bin: The `dbt` CLI. Defaults to `dbt`, so assumes it's on your + `PATH` :type dbt_bin: str :param verbose: The operator will log verbosely to the Airflow logs :type verbose: bool @@ -50,51 +57,105 @@ class DbtBaseOperator(BaseOperator): ui_color = '#d6522a' - template_fields = ['profiles_dir', 'project_dir', 'target', 'env', - 'vars', 'models', 'exclude', 'select', 'dbt_bin', 'verbose', - 'warn_error', 'full_refresh', 'data', 'schema', 'base_command'] + template_fields = ['env', 'dbt_bin', 'command', 'config'] @apply_defaults def __init__( self, - profiles_dir=None, - project_dir=None, - dir: str = None, - target=None, env: Dict = None, - vars=None, - models=None, - exclude=None, - select=None, dbt_bin='dbt', - verbose=True, - warn_error=False, - full_refresh=False, - data=False, - schema=False, dbt_hook=None, - base_command=None, - *args, - **kwargs): - super(DbtBaseOperator, self).__init__(*args, **kwargs) - - self.profiles_dir = profiles_dir - self.project_dir = project_dir if project_dir is not None else dir - self.target = target + command: Optional[str] = None, + config: DbtCommandConfig = None, + # if config was not provided we un-flatten them from the kwargs + # global flags + version: bool = False, + record_timing_info: bool = False, + debug: bool = False, + log_format: str = None, # either 'text', 'json' or 'default' + write_json: bool = None, + strict: bool = False, + warn_error: bool = False, + partial_parse: bool = False, + use_experimental_parser: bool = False, + use_colors: bool = None, + # command specific config + profiles_dir: str = None, + project_dir: str = None, + profile: str = None, + target: str = None, + config_dir: str = None, + resource_type: str = None, + vars: Dict = None, + # run specific + full_refresh: bool = False, + # ls specific + data: bool = False, + schema: bool = False, + models: str = None, + exclude: str = None, + select: str = None, + selector: str = None, + output: str = None, + output_keys: str = None, + # rpc specific + host: str = None, + port: str = None, + # test specific + fail_fast: bool = False, + args: dict = None, + no_compile=False, + *vargs, + **kwargs + ): + super(DbtBaseOperator, self).__init__(*vargs, **kwargs) self.env = {} if env is None else env - self.vars = vars - self.models = models - self.full_refresh = full_refresh - self.data = data - self.schema = schema - self.exclude = exclude - self.select = select self.dbt_bin = dbt_bin - self.verbose = verbose - self.warn_error = warn_error - self.base_command = base_command + self.dbt_command = command + self.dbt_command_config = config if config is not None else { + # global flags + 'version': version, + 'record_timing_info': record_timing_info, + 'debug': debug, + 'log_format': log_format, + 'write_json': write_json, + 'strict': strict, + 'warn_error': warn_error, + 'partial_parse': partial_parse, + 'use_experimental_parser': use_experimental_parser, + 'use_colors': use_colors, + # per command flags + 'profiles_dir': profiles_dir, + 'project_dir': project_dir, + 'target': target, + 'vars': vars, + # run specific + 'full_refresh': full_refresh, + 'profile': profile, + # docs specific + 'no_compile': no_compile, + # debug specific + 'config_dir': config_dir, + # ls specific + 'resource_type': resource_type, + 'select': select, + 'models': models, + 'exclude': exclude, + 'selector': selector, + 'output': output, + 'output_keys': output_keys, + # rpc specific + 'host': host, + 'port': port, + # run specific + 'fail_fast': fail_fast, + # run-operation specific + 'args': args, + # test specific + 'data': data, + 'schema': schema, + } self.hook = dbt_hook if dbt_hook is not None else DbtCliHook( - # dir=dir, env=self.env, dbt_bin=dbt_bin ) @@ -102,96 +163,103 @@ def __init__( def execute(self, context: Any): """Runs the provided command in the provided execution environment""" dbt_cli_command = self.hook.generate_dbt_cli_command( - base_command=self.base_command, - profiles_dir=self.profiles_dir, - project_dir=self.project_dir, - target=self.target, - vars=self.vars, - full_refresh=self.full_refresh, - data=self.data, - schema=self.schema, - models=self.models, - exclude=self.exclude, - select=self.select, - warn_error=self.warn_error, + dbt_bin=self.dbt_bin, + command=self.dbt_command, + **self.dbt_command_config ) - logging.info(f'Running dbt command "{dbt_cli_command}"') + command_str = '\n '.join(dbt_cli_command) + logging.info(f'Running dbt command "{command_str}"') self.hook.run_dbt(dbt_cli_command) class DbtRunOperator(DbtBaseOperator): """Runs a dbt run command""" + @apply_defaults - def __init__(self, profiles_dir=None, target=None, *args, **kwargs): - super().__init__( - profiles_dir=profiles_dir, - target=target, - base_command='run', - *args, - **kwargs - ) + def __init__(self, *args, **kwargs): + super().__init__(*args, command='run', **kwargs) class DbtTestOperator(DbtBaseOperator): """Runs a dbt test command""" + @apply_defaults - def __init__(self, profiles_dir=None, target=None, *args, **kwargs): - super().__init__( - profiles_dir=profiles_dir, - target=target, - base_command='test', - *args, - **kwargs - ) + def __init__(self, *args, **kwargs): + super().__init__(*args, command='test', **kwargs) class DbtDocsGenerateOperator(DbtBaseOperator): """Runs a dbt docs generate command""" + @apply_defaults - def __init__(self, profiles_dir=None, target=None, *args, **kwargs): - super().__init__( - profiles_dir=profiles_dir, - target=target, - base_command='docs generate', - *args, - **kwargs - ) + def __init__(self, *args, **kwargs): + super().__init__(*args, command='docs generate', **kwargs) class DbtSnapshotOperator(DbtBaseOperator): """Runs a dbt snapshot command""" + @apply_defaults - def __init__(self, profiles_dir=None, target=None, *args, **kwargs): - super().__init__( - profiles_dir=profiles_dir, - target=target, - base_command='snapshot', - *args, - **kwargs - ) + def __init__(self, *args, **kwargs): + super().__init__(*args, command='snapshot', **kwargs) class DbtSeedOperator(DbtBaseOperator): """Runs a dbt seed command""" + @apply_defaults - def __init__(self, profiles_dir=None, target=None, *args, **kwargs): - super().__init__( - profiles_dir=profiles_dir, - target=target, - base_command='seed', - *args, - **kwargs - ) + def __init__(self, *args, **kwargs): + super().__init__(*args, command='seed', **kwargs) class DbtDepsOperator(DbtBaseOperator): """Runs a dbt deps command""" + @apply_defaults - def __init__(self, profiles_dir=None, target=None, *args, **kwargs): - super().__init__( - profiles_dir=profiles_dir, - target=target, - base_command='deps', + def __init__(self, *args, **kwargs): + super().__init__(*args, command='deps', **kwargs) + + +class DbtCloudBuildOperator(DbtBaseOperator): + """Uses the CloudBuild Hook to run the operation in there by default""" + + template_fields = ['env', 'dbt_bin', 'command', 'config', + 'gcs_staging_location', 'project_id', 'dbt_version', 'service_account'] + + @apply_defaults + def __init__( + self, + gcs_staging_location: str, + env: Dict = None, + config: DbtCommandConfig = None, + project_id: str = None, + gcp_conn_id: str = None, + dbt_version: str = None, + service_account: str = None, + *args, + **kwargs + ): + + hook_config = { + 'env': env, + 'gcs_staging_location': gcs_staging_location, + } + if project_id is not None: + hook_config['project_id'] = project_id + if gcp_conn_id is not None: + hook_config['gcp_conn_id'] = gcp_conn_id + if dbt_version is not None: + hook_config['dbt_version'] = dbt_version + if service_account is not None: + hook_config['service_account'] = service_account + + self.hook = DbtCloudBuildHook(**hook_config) + + super(DbtCloudBuildOperator, self).__init__( + env=env, + command=None, + dbt_hook=self.hook, + config=config, *args, **kwargs ) From 457f916b25720bc075a477e8d22468fdc754c8cb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 26 Oct 2021 03:56:55 +0200 Subject: [PATCH 25/79] Remove dbt_bin from constructors and defer hook instantiation * dbt_bin is not needed other than to build the command, it's not necessary for the hook to have it in the constructor. * Operator templated fields are interpolated after the constructor runs. We're instantiating the hooks when the "executor" is called. Otherwise the hook has to be provided already instantiated. --- airflow_dbt/hooks/base.py | 3 +- airflow_dbt/hooks/cli.py | 11 ++-- airflow_dbt/hooks/google.py | 7 +-- airflow_dbt/operators/dbt_operator.py | 74 +++++++++++++++++---------- 4 files changed, 58 insertions(+), 37 deletions(-) diff --git a/airflow_dbt/hooks/base.py b/airflow_dbt/hooks/base.py index e9394d2..eeb9f09 100644 --- a/airflow_dbt/hooks/base.py +++ b/airflow_dbt/hooks/base.py @@ -18,10 +18,9 @@ class DbtBaseHook(BaseHook, ABC): :type dbt_bin: str """ - def __init__(self, env: Dict = None, dbt_bin='dbt'): + def __init__(self, env: Dict = None): super().__init__() self.env = env if env is not None else {} - self.dbt_bin = dbt_bin def generate_dbt_cli_command( self, diff --git a/airflow_dbt/hooks/cli.py b/airflow_dbt/hooks/cli.py index c177482..159993d 100644 --- a/airflow_dbt/hooks/cli.py +++ b/airflow_dbt/hooks/cli.py @@ -2,6 +2,7 @@ from typing import Any, Dict, List, Union +from airflow import AirflowException from airflow.hooks.subprocess import SubprocessHook from .base import DbtBaseHook @@ -23,9 +24,9 @@ class DbtCliHook(DbtBaseHook): :type dbt_bin: str """ - def __init__(self, env: Dict = None, dbt_bin='dbt'): + def __init__(self, env: Dict = None): self.sp = SubprocessHook() - super().__init__(env=env, dbt_bin=dbt_bin) + super().__init__(env=env) def get_conn(self) -> Any: """ @@ -41,11 +42,15 @@ def run_dbt(self, dbt_cmd: Union[str, List[str]]): :param dbt_cmd: The dbt whole command to run :type dbt_cmd: List[str] """ - self.sp.run_command( + result = self.sp.run_command( command=dbt_cmd, env=self.env, ) + if result.exit_code != 0: + raise AirflowException(f'Error executing the DBT command: ' + f'{result.output}') + def on_kill(self): """Kill the open subprocess if the task gets killed by Airflow""" self.sp.send_sigterm() diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index 4722319..7ded161 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -66,7 +66,7 @@ class DbtCloudBuildHook(DbtBaseHook): def __init__( self, - project_id: str, + project_id: str = None, # dir: str = None, gcs_staging_location: str = None, gcp_conn_id: str = "google_cloud_default", @@ -92,10 +92,7 @@ def __init__( self.project_id = project_id self.service_account = service_account - super().__init__( - env=env, - dbt_bin=dbt_bin - ) + super().__init__(env=env) def get_conn(self) -> Any: """Returns the cloud build connection, which is a gcp connection""" diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index a4caf9b..f1cb2c9 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -1,11 +1,10 @@ -import logging from typing import Any, Dict, Optional from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow_dbt.hooks import DbtCliHook, DbtCloudBuildHook from airflow_dbt.dbt_command_config import DbtCommandConfig +from airflow_dbt.hooks import DbtCliHook, DbtCloudBuildHook class DbtBaseOperator(BaseOperator): @@ -63,10 +62,11 @@ class DbtBaseOperator(BaseOperator): def __init__( self, env: Dict = None, - dbt_bin='dbt', + dbt_bin: Optional[str] = 'dbt', dbt_hook=None, command: Optional[str] = None, config: DbtCommandConfig = None, + # if config was not provided we un-flatten them from the kwargs # global flags version: bool = False, @@ -111,8 +111,8 @@ def __init__( super(DbtBaseOperator, self).__init__(*vargs, **kwargs) self.env = {} if env is None else env self.dbt_bin = dbt_bin - self.dbt_command = command - self.dbt_command_config = config if config is not None else { + self.command = command + self.config = config if config is not None else { # global flags 'version': version, 'record_timing_info': record_timing_info, @@ -155,20 +155,28 @@ def __init__( 'data': data, 'schema': schema, } + self.env = env + self.hook = dbt_hook + + def instantiate_hook(self): + """ + Instantiates the underlying dbt hook. This has to be deferred until + after the constructor or the templated params wont be interpolated. + """ + dbt_hook = self.hook self.hook = dbt_hook if dbt_hook is not None else DbtCliHook( env=self.env, - dbt_bin=dbt_bin ) def execute(self, context: Any): """Runs the provided command in the provided execution environment""" + self.instantiate_hook() + dbt_cli_command = self.hook.generate_dbt_cli_command( dbt_bin=self.dbt_bin, - command=self.dbt_command, - **self.dbt_command_config + command=self.command, + **self.config ) - command_str = '\n '.join(dbt_cli_command) - logging.info(f'Running dbt command "{command_str}"') self.hook.run_dbt(dbt_cli_command) @@ -226,6 +234,7 @@ class DbtCloudBuildOperator(DbtBaseOperator): template_fields = ['env', 'dbt_bin', 'command', 'config', 'gcs_staging_location', 'project_id', 'dbt_version', 'service_account'] + @apply_defaults def __init__( self, @@ -235,31 +244,42 @@ def __init__( project_id: str = None, gcp_conn_id: str = None, dbt_version: str = None, + dbt_bin: Optional[str] = None, service_account: str = None, *args, **kwargs ): - - hook_config = { - 'env': env, - 'gcs_staging_location': gcs_staging_location, - } - if project_id is not None: - hook_config['project_id'] = project_id - if gcp_conn_id is not None: - hook_config['gcp_conn_id'] = gcp_conn_id - if dbt_version is not None: - hook_config['dbt_version'] = dbt_version - if service_account is not None: - hook_config['service_account'] = service_account - - self.hook = DbtCloudBuildHook(**hook_config) + self.gcs_staging_location = gcs_staging_location + self.gcp_conn_id = gcp_conn_id + self.project_id = project_id + self.dbt_version = dbt_version + self.service_account = service_account super(DbtCloudBuildOperator, self).__init__( env=env, - command=None, - dbt_hook=self.hook, config=config, + dbt_bin=dbt_bin, *args, **kwargs ) + + def instantiate_hook(self): + """ + Instantiates a Cloud build dbt hook. This has to be done out of the + constructor because by the time the constructor runs the params have + not been yet interpolated. + """ + hook_config = { + 'env': self.env, + 'gcs_staging_location': self.gcs_staging_location, + } + if self.project_id is not None: + hook_config['project_id'] = self.project_id + if self.gcp_conn_id is not None: + hook_config['gcp_conn_id'] = self.gcp_conn_id + if self.dbt_version is not None: + hook_config['dbt_version'] = self.dbt_version + if self.service_account is not None: + hook_config['service_account'] = self.service_account + + self.hook = DbtCloudBuildHook(**hook_config) From 5c3159c45567a9a756f7ef0e96b1ef123cce3849 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 26 Oct 2021 10:45:48 +0200 Subject: [PATCH 26/79] Fix test for TestDbtCliHook checking return code different from 0 --- tests/hooks/test_dbt_hook.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/hooks/test_dbt_hook.py b/tests/hooks/test_dbt_hook.py index 199dd2d..51464d9 100644 --- a/tests/hooks/test_dbt_hook.py +++ b/tests/hooks/test_dbt_hook.py @@ -1,7 +1,7 @@ from unittest import TestCase, mock from unittest.mock import patch -from airflow.hooks.subprocess import SubprocessHook +from airflow.hooks.subprocess import SubprocessHook, SubprocessResult from airflow_dbt.hooks.cli import DbtCliHook from airflow_dbt.hooks.google import DbtCloudBuildHook @@ -10,6 +10,8 @@ class TestDbtCliHook(TestCase): @mock.patch.object(SubprocessHook, 'run_command') def test_sub_commands(self, mock_run_command): + mock_run_command.return_value = SubprocessResult( + exit_code=0, output='all good') hook = DbtCliHook() hook.run_dbt(['dbt', 'docs', 'generate']) mock_run_command.assert_called_once_with( From bec7fe5a2e5dcd202dbda572b6e0c7b041163bd4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 26 Oct 2021 11:06:12 +0200 Subject: [PATCH 27/79] Bring back "dir" to fix back the API, add warnings --- airflow_dbt/operators/dbt_operator.py | 22 +++++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index f1cb2c9..90f6ac2 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -1,3 +1,5 @@ +import logging +import warnings from typing import Any, Dict, Optional from airflow.models import BaseOperator @@ -66,7 +68,8 @@ def __init__( dbt_hook=None, command: Optional[str] = None, config: DbtCommandConfig = None, - + # dir deprecated in favor of dbt native project and profile directories + dir: str = None, # if config was not provided we un-flatten them from the kwargs # global flags version: bool = False, @@ -105,10 +108,19 @@ def __init__( fail_fast: bool = False, args: dict = None, no_compile=False, + *vargs, **kwargs ): super(DbtBaseOperator, self).__init__(*vargs, **kwargs) + + if dir is not None: + warnings.warn('"dir" param is deprecated in favor of dbt native ' + 'param "project_dir"') + if project_dir is None: + logging.warning('Using "dir" as "project_dir"') + project_dir = dir + self.env = {} if env is None else env self.dbt_bin = dbt_bin self.command = command @@ -228,6 +240,14 @@ def __init__(self, *args, **kwargs): super().__init__(*args, command='deps', **kwargs) +class DbtCleanOperator(DbtBaseOperator): + """Runs a dbt clean command""" + + @apply_defaults + def __init__(self, *args, **kwargs): + super().__init__(*args, command='clean', **kwargs) + + class DbtCloudBuildOperator(DbtBaseOperator): """Uses the CloudBuild Hook to run the operation in there by default""" From 0c7587aa79d9e5ac9219f8fd1746aadc71e077d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 26 Oct 2021 11:24:35 +0200 Subject: [PATCH 28/79] DbtCleanOperator was missing from operator exports (__init__) --- airflow_dbt/operators/__init__.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow_dbt/operators/__init__.py b/airflow_dbt/operators/__init__.py index 5b75e29..61373dc 100644 --- a/airflow_dbt/operators/__init__.py +++ b/airflow_dbt/operators/__init__.py @@ -6,5 +6,6 @@ DbtTestOperator, DbtDocsGenerateOperator, DbtDepsOperator, + DbtCleanOperator, DbtCloudBuildOperator ) From ce732b2e16bd9a4e4fc39543147ff1f384b7c7fa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 26 Oct 2021 15:16:01 +0200 Subject: [PATCH 29/79] Fix typo (thanks @fenimore) and reorganize imports --- airflow_dbt/hooks/google.py | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index 7ded161..6ef5c61 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -1,15 +1,10 @@ import logging -import os import pprint -import tarfile -from tempfile import NamedTemporaryFile from typing import Any, Dict, List from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.cloud_build import CloudBuildHook -from airflow.providers.google.cloud.hooks.gcs import ( - GCSHook, _parse_gcs_url, -) +from airflow.providers.google.cloud.hooks.gcs import _parse_gcs_url from airflow.providers.google.get_provider_info import get_provider_info from packaging import version @@ -24,7 +19,7 @@ v_provider = version.parse(google_providers_version) if not v_min <= v_provider < v_max: raise Exception( - f'The provider "apache-airflow-providers-google" version "' + 'The provider "apache-airflow-providers-google" version "' f'{google_providers_version}" is not compatible with the current API. ' f'Please install a compatible version in the range [{v_min}, {v_max})"' ) From 8ea99586a66e7abbb7b65caf07facf8e395d9897 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 26 Oct 2021 15:31:41 +0200 Subject: [PATCH 30/79] Comply with linter (flake8) --- airflow_dbt/hooks/google.py | 2 +- airflow_dbt/operators/dbt_operator.py | 7 ++++--- tests/hooks/test_dbt_hook.py | 9 +++++---- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index 6ef5c61..ccdb605 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -76,7 +76,7 @@ def __init__( if not staging_blob.endswith('.tar.gz'): raise AirflowException( f'The provided blob "{staging_blob}" to a compressed file does not ' + - f'have the right extension ".tar.gz' + 'have the right extension ".tar.gz' ) self.gcs_staging_bucket = staging_bucket self.gcs_staging_blob = staging_blob diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 90f6ac2..f2b9751 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -251,9 +251,10 @@ def __init__(self, *args, **kwargs): class DbtCloudBuildOperator(DbtBaseOperator): """Uses the CloudBuild Hook to run the operation in there by default""" - template_fields = ['env', 'dbt_bin', 'command', 'config', - 'gcs_staging_location', 'project_id', 'dbt_version', 'service_account'] - + template_fields = [ + 'env', 'dbt_bin', 'command', 'config', 'gcs_staging_location', + 'project_id', 'dbt_version', 'service_account' + ] @apply_defaults def __init__( diff --git a/tests/hooks/test_dbt_hook.py b/tests/hooks/test_dbt_hook.py index 51464d9..9c1cd5d 100644 --- a/tests/hooks/test_dbt_hook.py +++ b/tests/hooks/test_dbt_hook.py @@ -27,8 +27,9 @@ def test_vars(self): vars={"foo": "bar", "baz": "true"} ) - assert generated_command == ['dbt', 'run', '--vars', - '{"foo": "bar", "baz": "true"}'] + assert generated_command == [ + 'dbt', 'run', '--vars', '{"foo": "bar", "baz": "true"}' + ] class TestDbtCloudBuildHook(TestCase): @@ -46,7 +47,7 @@ def test_create_build(self, MockCloudBuildHook): expected_body = { 'steps': [{ - 'name': f'fishtownanalytics/dbt:0.10.10', + 'name': 'fishtownanalytics/dbt:0.10.10', 'args': ['docs', 'generate'], 'env': ['TEST_ENV_VAR=test'] }], @@ -62,4 +63,4 @@ def test_create_build(self, MockCloudBuildHook): mock_create_build.assert_called_once_with( body=expected_body, project_id='test_project_id' - ) \ No newline at end of file + ) From f1c84fe065354d5693a6a04a75b0823af7d30ae3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Wed, 27 Oct 2021 14:35:45 +0200 Subject: [PATCH 31/79] Refactor, move google operator out of the dbt_operator This way if the extra is not installed the google dependencies are not imported and it doesn't create an error --- airflow_dbt/__init__.py | 16 ++++- airflow_dbt/operators/__init__.py | 7 ++- airflow_dbt/operators/dbt_operator.py | 60 +------------------ airflow_dbt/operators/google.py | 84 +++++++++++++++++++++++++++ 4 files changed, 105 insertions(+), 62 deletions(-) create mode 100644 airflow_dbt/operators/google.py diff --git a/airflow_dbt/__init__.py b/airflow_dbt/__init__.py index 9419f4f..b1f6dac 100644 --- a/airflow_dbt/__init__.py +++ b/airflow_dbt/__init__.py @@ -1,9 +1,21 @@ -from .hooks import DbtCliHook +from .hooks import ( + DbtCloudBuildHook, + DbtBaseHook, + DbtCliHook, +) + from .operators import ( + DbtBaseOperator, DbtSeedOperator, DbtSnapshotOperator, DbtRunOperator, DbtTestOperator, DbtDocsGenerateOperator, - DbtDepsOperator + DbtDepsOperator, + DbtCleanOperator, + DbtCloudBuildOperator, + CloudBuildLogsLink, + CloudBuildLinkPlugin, ) + +from dbt_command_config import DbtCommandConfig diff --git a/airflow_dbt/operators/__init__.py b/airflow_dbt/operators/__init__.py index 61373dc..ac6368e 100644 --- a/airflow_dbt/operators/__init__.py +++ b/airflow_dbt/operators/__init__.py @@ -7,5 +7,10 @@ DbtDocsGenerateOperator, DbtDepsOperator, DbtCleanOperator, - DbtCloudBuildOperator +) + +from .google import ( + DbtCloudBuildOperator, + CloudBuildLogsLink, + CloudBuildLinkPlugin, ) diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index f2b9751..b584cdb 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -6,7 +6,7 @@ from airflow.utils.decorators import apply_defaults from airflow_dbt.dbt_command_config import DbtCommandConfig -from airflow_dbt.hooks import DbtCliHook, DbtCloudBuildHook +from airflow_dbt.hooks import DbtCliHook class DbtBaseOperator(BaseOperator): @@ -246,61 +246,3 @@ class DbtCleanOperator(DbtBaseOperator): @apply_defaults def __init__(self, *args, **kwargs): super().__init__(*args, command='clean', **kwargs) - - -class DbtCloudBuildOperator(DbtBaseOperator): - """Uses the CloudBuild Hook to run the operation in there by default""" - - template_fields = [ - 'env', 'dbt_bin', 'command', 'config', 'gcs_staging_location', - 'project_id', 'dbt_version', 'service_account' - ] - - @apply_defaults - def __init__( - self, - gcs_staging_location: str, - env: Dict = None, - config: DbtCommandConfig = None, - project_id: str = None, - gcp_conn_id: str = None, - dbt_version: str = None, - dbt_bin: Optional[str] = None, - service_account: str = None, - *args, - **kwargs - ): - self.gcs_staging_location = gcs_staging_location - self.gcp_conn_id = gcp_conn_id - self.project_id = project_id - self.dbt_version = dbt_version - self.service_account = service_account - - super(DbtCloudBuildOperator, self).__init__( - env=env, - config=config, - dbt_bin=dbt_bin, - *args, - **kwargs - ) - - def instantiate_hook(self): - """ - Instantiates a Cloud build dbt hook. This has to be done out of the - constructor because by the time the constructor runs the params have - not been yet interpolated. - """ - hook_config = { - 'env': self.env, - 'gcs_staging_location': self.gcs_staging_location, - } - if self.project_id is not None: - hook_config['project_id'] = self.project_id - if self.gcp_conn_id is not None: - hook_config['gcp_conn_id'] = self.gcp_conn_id - if self.dbt_version is not None: - hook_config['dbt_version'] = self.dbt_version - if self.service_account is not None: - hook_config['service_account'] = self.service_account - - self.hook = DbtCloudBuildHook(**hook_config) diff --git a/airflow_dbt/operators/google.py b/airflow_dbt/operators/google.py new file mode 100644 index 0000000..8a3ef7c --- /dev/null +++ b/airflow_dbt/operators/google.py @@ -0,0 +1,84 @@ +from typing import Dict, Optional + +from airflow.models import BaseOperatorLink +from airflow.plugins_manager import AirflowPlugin +from airflow.utils.decorators import apply_defaults + +from airflow_dbt.operators import DbtBaseOperator +from airflow_dbt.dbt_command_config import DbtCommandConfig +from airflow_dbt.hooks import DbtCloudBuildHook + + +class CloudBuildLogsLink(BaseOperatorLink): + """Add a link to the logs generated from a build in cloud build""" + name = "Cloud Build Logs" + + def get_link(self, operator, _): + """Returns the log url for the cloud build logs stored as class prop""" + return operator.log_url + + +class CloudBuildLinkPlugin(AirflowPlugin): + """Registers the extra links""" + name = "cloud_build_link_plugin" + operator_extra_links = [CloudBuildLogsLink()] + + +class DbtCloudBuildOperator(DbtBaseOperator): + """Uses the CloudBuild Hook to run the operation in there by default""" + + template_fields = [ + 'env', 'dbt_bin', 'command', 'config', 'gcs_staging_location', + 'project_id', 'dbt_version', 'service_account' + ] + + operator_extra_links = [CloudBuildLogsLink] + + @apply_defaults + def __init__( + self, + gcs_staging_location: str, + env: Dict = None, + config: DbtCommandConfig = None, + project_id: str = None, + gcp_conn_id: str = None, + dbt_version: str = None, + dbt_bin: Optional[str] = None, + service_account: str = None, + *args, + **kwargs + ): + self.gcs_staging_location = gcs_staging_location + self.gcp_conn_id = gcp_conn_id + self.project_id = project_id + self.dbt_version = dbt_version + self.service_account = service_account + + super(DbtCloudBuildOperator, self).__init__( + env=env, + config=config, + dbt_bin=dbt_bin, + *args, + **kwargs + ) + + def instantiate_hook(self): + """ + Instantiates a Cloud build dbt hook. This has to be done out of the + constructor because by the time the constructor runs the params have + not been yet interpolated. + """ + hook_config = { + 'env': self.env, + 'gcs_staging_location': self.gcs_staging_location, + } + if self.project_id is not None: + hook_config['project_id'] = self.project_id + if self.gcp_conn_id is not None: + hook_config['gcp_conn_id'] = self.gcp_conn_id + if self.dbt_version is not None: + hook_config['dbt_version'] = self.dbt_version + if self.service_account is not None: + hook_config['service_account'] = self.service_account + + self.hook = DbtCloudBuildHook(**hook_config) From a3e1213c2d26c2f8ce9aa5c4445ceda61f7ee410 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Wed, 27 Oct 2021 14:39:40 +0200 Subject: [PATCH 32/79] Display Cloud Build logs in the Airflow logs It retrieves them after the build is complete. As composer latest version of the providers is 5.1.0 we cannot stream them. We have to wait for the build to finish so it returns the build_id that we need to retrieve the build log. Also, logs are placed in the same folder as the zipped dbt artifact --- airflow_dbt/hooks/google.py | 43 +++++++++++++++++++++++++++---------- 1 file changed, 32 insertions(+), 11 deletions(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index ccdb605..badf24f 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -1,11 +1,11 @@ import logging -import pprint from typing import Any, Dict, List from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.cloud_build import CloudBuildHook -from airflow.providers.google.cloud.hooks.gcs import _parse_gcs_url +from airflow.providers.google.cloud.hooks.gcs import GCSHook, _parse_gcs_url from airflow.providers.google.get_provider_info import get_provider_info +from airflow.settings import json from packaging import version from .base import DbtBaseHook @@ -113,23 +113,44 @@ def run_dbt(self, dbt_cmd: List[str]): "bucket": self.gcs_staging_bucket, "object": self.gcs_staging_blob, } - } + }, + 'options': { + 'logging': 'GCS_ONLY', + }, + 'logsBucket': self.gcs_staging_bucket, } if self.service_account is not None: - cloud_build_config['serviceAccount'] = self.service_account + sa_path = f'projects/{self.project_id}/serviceAccounts/' \ + f'{self.service_account}' + cloud_build_config['serviceAccount'] = sa_path - cloud_build_config_str = pprint.pformat(cloud_build_config) - logging.info(f'Running the following cloud build config:\n{cloud_build_config_str}') + cloud_build_config_str = json.dumps(cloud_build_config, indent=2) + logging.info( + f'Running the following cloud build' + f' config:\n{cloud_build_config_str}' + ) - results = self.cloud_build_hook.create_build( + build_results = self.cloud_build_hook.create_build( body=cloud_build_config, project_id=self.project_id, ) - logging.info( - f'Triggered build {results["id"]}. You can find the logs at ' - f'{results["logUrl"]}' - ) + + # print logs from GCS + build_logs_blob = f'log-{build_results["id"]}.txt' + with GCSHook().provide_file( + bucket_name=self.gcs_staging_bucket, + object_name=build_logs_blob + ) as log_file_handle: + for line in log_file_handle: + clean_line = line.decode('utf-8').strip() + if not clean_line == '': + logging.info(clean_line) + + # print result from build + logging.info('Build results:\n' + json.dumps(build_results, indent=2)) + # set the log_url class param to be read from the "links" + self.log_url = build_results['logUrl'] def on_kill(self): """Stopping the build is not implemented until google providers v6""" From 7776c45937ef20cbd3850b11a3ebd6b62e8bca0f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Wed, 27 Oct 2021 15:22:39 +0200 Subject: [PATCH 33/79] Get rid of __init__ files exporting classes from different places I reached a couple of inconsistencies in the imports in the tests and in dev dags. Having a single path to export them will make them resolve uniquely --- airflow_dbt/__init__.py | 21 --------------------- airflow_dbt/hooks/__init__.py | 3 --- airflow_dbt/hooks/cli.py | 2 +- airflow_dbt/hooks/google.py | 2 +- airflow_dbt/operators/__init__.py | 16 ---------------- airflow_dbt/operators/dbt_operator.py | 2 +- airflow_dbt/operators/google.py | 4 ++-- 7 files changed, 5 insertions(+), 45 deletions(-) delete mode 100644 airflow_dbt/__init__.py delete mode 100644 airflow_dbt/hooks/__init__.py delete mode 100644 airflow_dbt/operators/__init__.py diff --git a/airflow_dbt/__init__.py b/airflow_dbt/__init__.py deleted file mode 100644 index b1f6dac..0000000 --- a/airflow_dbt/__init__.py +++ /dev/null @@ -1,21 +0,0 @@ -from .hooks import ( - DbtCloudBuildHook, - DbtBaseHook, - DbtCliHook, -) - -from .operators import ( - DbtBaseOperator, - DbtSeedOperator, - DbtSnapshotOperator, - DbtRunOperator, - DbtTestOperator, - DbtDocsGenerateOperator, - DbtDepsOperator, - DbtCleanOperator, - DbtCloudBuildOperator, - CloudBuildLogsLink, - CloudBuildLinkPlugin, -) - -from dbt_command_config import DbtCommandConfig diff --git a/airflow_dbt/hooks/__init__.py b/airflow_dbt/hooks/__init__.py deleted file mode 100644 index b08db3d..0000000 --- a/airflow_dbt/hooks/__init__.py +++ /dev/null @@ -1,3 +0,0 @@ -from .base import DbtBaseHook -from .cli import DbtCliHook -from .google import DbtCloudBuildHook diff --git a/airflow_dbt/hooks/cli.py b/airflow_dbt/hooks/cli.py index 159993d..6ddc280 100644 --- a/airflow_dbt/hooks/cli.py +++ b/airflow_dbt/hooks/cli.py @@ -5,7 +5,7 @@ from airflow import AirflowException from airflow.hooks.subprocess import SubprocessHook -from .base import DbtBaseHook +from airflow_dbt.hooks.base import DbtBaseHook class DbtCliHook(DbtBaseHook): diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index badf24f..2055401 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -8,7 +8,7 @@ from airflow.settings import json from packaging import version -from .base import DbtBaseHook +from airflow_dbt.hooks.base import DbtBaseHook # Check we're using the right google provider version. As composer is the # most brad used Airflow installation we will default to the latest version diff --git a/airflow_dbt/operators/__init__.py b/airflow_dbt/operators/__init__.py deleted file mode 100644 index ac6368e..0000000 --- a/airflow_dbt/operators/__init__.py +++ /dev/null @@ -1,16 +0,0 @@ -from .dbt_operator import ( - DbtBaseOperator, - DbtSeedOperator, - DbtSnapshotOperator, - DbtRunOperator, - DbtTestOperator, - DbtDocsGenerateOperator, - DbtDepsOperator, - DbtCleanOperator, -) - -from .google import ( - DbtCloudBuildOperator, - CloudBuildLogsLink, - CloudBuildLinkPlugin, -) diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index b584cdb..46e9be0 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -6,7 +6,7 @@ from airflow.utils.decorators import apply_defaults from airflow_dbt.dbt_command_config import DbtCommandConfig -from airflow_dbt.hooks import DbtCliHook +from airflow_dbt.hooks.cli import DbtCliHook class DbtBaseOperator(BaseOperator): diff --git a/airflow_dbt/operators/google.py b/airflow_dbt/operators/google.py index 8a3ef7c..add8f32 100644 --- a/airflow_dbt/operators/google.py +++ b/airflow_dbt/operators/google.py @@ -4,9 +4,9 @@ from airflow.plugins_manager import AirflowPlugin from airflow.utils.decorators import apply_defaults -from airflow_dbt.operators import DbtBaseOperator +from airflow_dbt.operators.dbt_operator import DbtBaseOperator from airflow_dbt.dbt_command_config import DbtCommandConfig -from airflow_dbt.hooks import DbtCloudBuildHook +from airflow_dbt.hooks.google import DbtCloudBuildHook class CloudBuildLogsLink(BaseOperatorLink): From 2f8b9e7a88af474f0dad7b580771c74b00fa2b5a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Wed, 27 Oct 2021 15:24:01 +0200 Subject: [PATCH 34/79] Fix TestDbtCloudBuildHook with the new config sent to Cloud Build --- tests/hooks/test_dbt_hook.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/tests/hooks/test_dbt_hook.py b/tests/hooks/test_dbt_hook.py index 9c1cd5d..7c1908e 100644 --- a/tests/hooks/test_dbt_hook.py +++ b/tests/hooks/test_dbt_hook.py @@ -34,8 +34,11 @@ def test_vars(self): class TestDbtCloudBuildHook(TestCase): @patch('airflow_dbt.hooks.google.CloudBuildHook') - def test_create_build(self, MockCloudBuildHook): + @patch('airflow_dbt.hooks.google.GCSHook') + def test_create_build(self, _, MockCloudBuildHook): mock_create_build = MockCloudBuildHook().create_build + mock_create_build.return_value = {'id': 'test_id', 'logUrl': + 'http://testurl.com'} hook = DbtCloudBuildHook( project_id='test_project_id', gcs_staging_location='gs://hello/file.tar.gz', @@ -57,7 +60,12 @@ def test_create_build(self, MockCloudBuildHook): 'object': 'file.tar.gz', } }, - 'serviceAccount': 'robot@mail.com' + 'serviceAccount': 'projects/test_project_id/serviceAccounts/robot@mail.com', + 'options': { + 'logging': 'GCS_ONLY', + + }, + 'logsBucket': 'hello', } mock_create_build.assert_called_once_with( From 3c90bc5d198e796c26c4d71487cddc648f3bab88 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Wed, 27 Oct 2021 17:51:51 +0200 Subject: [PATCH 35/79] Remove unused DbtCloudBuildHook param `dbt_bin` --- airflow_dbt/hooks/google.py | 1 - 1 file changed, 1 deletion(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index 2055401..f5ca8cd 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -67,7 +67,6 @@ def __init__( gcp_conn_id: str = "google_cloud_default", dbt_version: str = '0.21.0', env: Dict = None, - dbt_bin=None, service_account=None, ): staging_bucket, staging_blob = _parse_gcs_url(gcs_staging_location) From abd4bfefc75bafbdbbcec8ab0b605dde1cc6bb3f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 2 Nov 2021 14:38:54 +0100 Subject: [PATCH 36/79] Test extensively command generation, execution in cli, hook injection... --- tests/hooks/test_dbt_hook.py | 169 ++++++++++++++++++++++++--- tests/operators/test_dbt_operator.py | 147 ++++++++++++++--------- 2 files changed, 244 insertions(+), 72 deletions(-) diff --git a/tests/hooks/test_dbt_hook.py b/tests/hooks/test_dbt_hook.py index 7c1908e..0c73d75 100644 --- a/tests/hooks/test_dbt_hook.py +++ b/tests/hooks/test_dbt_hook.py @@ -1,35 +1,171 @@ +from typing import Union from unittest import TestCase, mock from unittest.mock import patch +import pytest +from airflow import AirflowException from airflow.hooks.subprocess import SubprocessHook, SubprocessResult +from pytest import mark +from airflow_dbt.dbt_command_config import DbtCommandConfig +from airflow_dbt.hooks.base import generate_dbt_cli_command from airflow_dbt.hooks.cli import DbtCliHook from airflow_dbt.hooks.google import DbtCloudBuildHook +@mark.parametrize( + ["dbt_bin", "command", "params", "expected_command"], [ + ("dbt", "run", {}, ["dbt", "run"]), + # check it runs with empty params + ("dbt", None, {}, ValueError()), # check it fails with no command + (None, "run", {}, ValueError()), # check it fails with no dbt_bin + ("dbt", "test", {}, ["dbt", "test"]), # test without params + ("dbt", "test", {'non_existing_param'}, TypeError()), + # test invalid param + ("dbt", "test", {'--models': None}, ValueError()), + # test mandatory value + ("dbt", "test", {'--models': 3}, ValueError()), # test wrong type + ("/bin/dbt", "test", {}, ["/bin/dbt", "test"]), # test dbt path + ("dbt", "run", {'full_refresh': False}, ValueError()), + # test flags always positive + ('/home/airflow/.local/bin/dbt', 'run', { + 'full_refresh': True, + 'profiles_dir': '/opt/airflow/dags/dbt_project', + 'project_dir': '/opt/airflow/dags/project_dir', + 'vars': {'execution_date': '2021-01-01'}, + 'select': 'my_model', + }, ['/home/airflow/.local/bin/dbt', 'run', '--full-refresh', + '--profiles-dir', '/opt/airflow/dags/dbt_project', + '--project-dir', '/opt/airflow/dags/project_dir', + '--vars', '{"execution_date": "2021-01-01"}', '--select', + 'my_model']), + # test all the params + ("dbt", "test", {'profiles_dir': '/path/profiles_folder'}, + ["dbt", "test", "--profiles-dir", "/path/profiles_folder"]), + ("dbt", "run", {'project_dir': '/path/dbt_project_dir'}, + ["dbt", "run", "--project-dir", "/path/dbt_project_dir"]), + ("dbt", "test", {'target': 'model_target'}, + ["dbt", "test", "--target", "model_target"]), + ("dbt", "test", {'vars': {"hello": "world"}}, + ["dbt", "test", "--vars", '{"hello": "world"}']), + ("dbt", "run", {'models': 'my_model'}, + ["dbt", "run", "--models", "my_model"]), + ("dbt", "run", {'exclude': 'my_model'}, + ["dbt", "run", "--exclude", "my_model"]), + ("dbt", "run", {'exclude': 'my_model'}, + ["dbt", "run", "--exclude", "my_model"]), + + # run specific params + ("dbt", "run", {'full_refresh': True}, + ["dbt", "run", "--full-refresh"]), + ("dbt", "run", {'full_refresh': 3}, TypeError()), + ("dbt", "run", {'full_refresh': 'hello'}, TypeError()), + ("dbt", "run", {'profile': 'test_profile'}, + ["dbt", "run", "--profile", "test_profile"]), + + # docs specific params + ("dbt", "docs", {'no_compile': True}, + ["dbt", "docs", "--no-compile"]), + + # debug specific params + ("dbt", "debug", {'config_dir': '/path/to/config_dir'}, + ["dbt", "debug", "--config-dir", '/path/to/config_dir']), + + # ls specific params + ("dbt", "ls", {'resource_type': '/path/to/config_dir'}, + ["dbt", "ls", "--resource-type", '/path/to/config_dir']), + ("dbt", "ls", {'select': 'my_model'}, + ["dbt", "ls", "--select", "my_model"]), + ("dbt", "ls", {'exclude': 'my_model'}, + ["dbt", "ls", "--exclude", "my_model"]), + ("dbt", "ls", {'output': 'my_model'}, + ["dbt", "ls", "--output", "my_model"]), + ("dbt", "ls", {'output_keys': 'my_model'}, + ["dbt", "ls", "--output-keys", "my_model"]), + + # rpc specific params + ("dbt", "rpc", {'host': 'http://my-host-url.com'}, + ["dbt", "rpc", "--host", 'http://my-host-url.com']), + ("dbt", "rpc", {'port': '8080'}, TypeError()), + ("dbt", "rpc", {'port': 8080}, ["dbt", "rpc", "--port", '8080']), + + # run specific params + ("dbt", "run", {'fail_fast': True}, ["dbt", "run", "--fail-fast"]), + + # test specific params + ("dbt", "test", {'data': True}, ["dbt", "test", '--data']), + ("dbt", "test", {'schema': True}, ["dbt", "test", '--schema']), + # without params + + ] +) +def test_create_cli_command_from_params( + dbt_bin: str, + command: str, + params: DbtCommandConfig, + expected_command: Union[list[str], Exception] +): + """ + Test that the function create_cli_command_from_params returns the + correct + command or raises the correct exception + :type expected_command: object + """ + if isinstance(expected_command, Exception): + with pytest.raises(expected_command.__class__): + generate_dbt_cli_command(dbt_bin, command, **params) + else: + assert generate_dbt_cli_command(dbt_bin, command, **params) \ + == expected_command + + class TestDbtCliHook(TestCase): - @mock.patch.object(SubprocessHook, 'run_command') + @mock.patch.object( + SubprocessHook, + 'run_command', + return_value=SubprocessResult(exit_code=0, output='all good') + ) def test_sub_commands(self, mock_run_command): - mock_run_command.return_value = SubprocessResult( - exit_code=0, output='all good') - hook = DbtCliHook() + """ + Test that sub commands are called with the right params + """ + hook = DbtCliHook(env={'GOOGLE_APPLICATION_CREDENTIALS': 'my_creds'}) hook.run_dbt(['dbt', 'docs', 'generate']) mock_run_command.assert_called_once_with( command=['dbt', 'docs', 'generate'], - env={} + env={'GOOGLE_APPLICATION_CREDENTIALS': 'my_creds'} ) - def test_vars(self): + @mock.patch.object( + SubprocessHook, + 'run_command', + return_value=SubprocessResult(exit_code=1, output='some error') + ) + def test_run_dbt(self, mock_run_command): + """ + Patch SubProcessHook to return a non-0 exit code and check we raise + an exception for such a result + """ + + with pytest.raises(AirflowException): + hook = DbtCliHook(env={'GOOGLE_APPLICATION_CREDENTIALS': 'my_creds'}) + hook.run_dbt(['dbt', 'run']) + mock_run_command.assert_called_once_with( + command=['dbt', 'run'], + env={'GOOGLE_APPLICATION_CREDENTIALS': 'my_creds'} + ) + + @mock.patch.object(SubprocessHook, 'get_conn') + def test_subprocess_kill_called(self, mock_get_conn): hook = DbtCliHook() - generated_command = hook.generate_dbt_cli_command( - dbt_bin='dbt', - command='run', - vars={"foo": "bar", "baz": "true"} - ) + hook.get_conn() + mock_get_conn.assert_called_once() - assert generated_command == [ - 'dbt', 'run', '--vars', '{"foo": "bar", "baz": "true"}' - ] + @mock.patch.object(SubprocessHook, 'send_sigterm') + def test_subprocess_get_conn_called(self, mock_send_sigterm): + hook = DbtCliHook() + hook.on_kill() + mock_send_sigterm.assert_called_once() class TestDbtCloudBuildHook(TestCase): @@ -37,8 +173,9 @@ class TestDbtCloudBuildHook(TestCase): @patch('airflow_dbt.hooks.google.GCSHook') def test_create_build(self, _, MockCloudBuildHook): mock_create_build = MockCloudBuildHook().create_build - mock_create_build.return_value = {'id': 'test_id', 'logUrl': - 'http://testurl.com'} + mock_create_build.return_value = { + 'id': 'test_id', 'logUrl': 'http://testurl.com' + } hook = DbtCloudBuildHook( project_id='test_project_id', gcs_staging_location='gs://hello/file.tar.gz', diff --git a/tests/operators/test_dbt_operator.py b/tests/operators/test_dbt_operator.py index 48cd1ba..52d5a0d 100644 --- a/tests/operators/test_dbt_operator.py +++ b/tests/operators/test_dbt_operator.py @@ -1,75 +1,110 @@ import datetime -from unittest import TestCase, mock +from typing import Union +from unittest.mock import MagicMock, patch +import pytest from airflow import DAG, configuration +from pytest import fixture, mark from airflow_dbt.hooks.cli import DbtCliHook +from airflow_dbt.hooks.google import DbtCloudBuildHook from airflow_dbt.operators.dbt_operator import ( - DbtDepsOperator, DbtRunOperator, DbtSeedOperator, DbtSnapshotOperator, + DbtBaseOperator, + DbtCleanOperator, + DbtDepsOperator, + DbtDocsGenerateOperator, + DbtRunOperator, + DbtSeedOperator, + DbtSnapshotOperator, DbtTestOperator, ) +from airflow_dbt.operators.google import DbtCloudBuildOperator -class TestDbtCliOperator(TestCase): - def setUp(self): - configuration.conf.load_test_config() - args = { - 'owner': 'airflow', - 'start_date': datetime.datetime(2020, 2, 27) - } - self.dag = DAG('test_dag_id', default_args=args) +@fixture +def airflow_dag(): + """Instantiates an Airflow DAG to be used as a test fixture""" + configuration.conf.load_test_config() + args = { + 'owner': 'airflow', + 'start_date': datetime.datetime(2020, 2, 27) + } + yield DAG('test_dag_id', default_args=args) - @mock.patch.object(DbtCliHook, 'run_dbt') - def test_dbt_run(self, mock_run_dbt): - operator = DbtRunOperator( - task_id='run', - dag=self.dag - ) - operator.execute(None) - mock_run_dbt.assert_called_once_with(['dbt', 'run']) - @mock.patch.object(DbtCliHook, 'run_dbt') - def test_dbt_test(self, mock_run_dbt): - operator = DbtTestOperator( - task_id='test', - dag=self.dag - ) +@mark.parametrize( + ['Operator', 'expected_command'], [ + (DbtBaseOperator, ValueError()), + (DbtDepsOperator, ['dbt', 'deps']), + (DbtRunOperator, ['dbt', 'run']), + (DbtSeedOperator, ['dbt', 'seed']), + (DbtDocsGenerateOperator, ['dbt', 'docs generate']), + (DbtSnapshotOperator, ['dbt', 'snapshot']), + (DbtCleanOperator, ['dbt', 'clean']), + (DbtTestOperator, ['dbt', 'test']), + ] +) +@patch.object(DbtCliHook, 'run_dbt') +def test_basic_dbt_operators( + mock_run_dbt: MagicMock, + Operator: DbtBaseOperator, + expected_command: Union[list[str], Exception], + airflow_dag: DAG, +): + """ + Test that all the basic Dbt{Command}Operators instantiate the right + default dbt command. And that the basic DbtBaseOperator raises a value + Error since there's no base command defined to be executed + command + """ + # noinspection PyCallingNonCallable + operator = Operator( + task_id=f'{Operator.__name__}', + dag=airflow_dag + ) + if isinstance(expected_command, Exception): + with pytest.raises(expected_command.__class__): + operator.execute(None) + else: operator.execute(None) - mock_run_dbt.assert_called_once_with(['dbt', 'test']) + mock_run_dbt.assert_called_once_with(expected_command) - @mock.patch.object(DbtCliHook, 'run_dbt') - def test_dbt_snapshot(self, mock_run_dbt): - operator = DbtSnapshotOperator( - task_id='snapshot', - dag=self.dag - ) - operator.execute(None) - mock_run_dbt.assert_called_once_with(['dbt', 'snapshot']) - @mock.patch.object(DbtCliHook, 'run_dbt') - def test_dbt_seed(self, mock_run_dbt): - operator = DbtSeedOperator( - task_id='seed', - dag=self.dag +def test_dbt_warns_about_dir_param(airflow_dag: DAG): + """ + Test that the DbtBaseOperator warns about the use of the dir parameter + """ + with pytest.warns(PendingDeprecationWarning): + DbtBaseOperator( + task_id='test_task_id', + dag=airflow_dag, + dir='/tmp/dbt' ) - operator.execute(None) - mock_run_dbt.assert_called_once_with(['dbt', 'seed']) - @mock.patch.object(DbtCliHook, 'run_dbt') - def test_dbt_deps(self, mock_run_dbt): - operator = DbtDepsOperator( - task_id='deps', - dag=self.dag - ) - operator.execute(None) - mock_run_dbt.assert_called_once_with(['dbt', 'deps']) +@patch.object(DbtCloudBuildHook, '__init__', return_value=None) +def test_cloud_build_operator_instantiates_hook( + cloud_build_hook_constructor: MagicMock, + airflow_dag: DAG +): + hook = DbtCloudBuildOperator( + task_id='test_cloud_build', + gcs_staging_location='gs://my_bucket/dbt_proj.tar.gz', + env={'CONFIG_VAR': 'HELLO'}, + config={'project_dir': 'not used'}, + project_id='my_project', + gcp_conn_id='test_gcp_conn', + dbt_version='0.19.2', + service_account='dbt-sa@google.com', + dag=airflow_dag + ) + hook.instantiate_hook() -class TestDbtRunWithCloudBuild(TestCase): - def setUp(self): - configuration.conf.load_test_config() - args = { - 'owner': 'airflow', - 'start_date': datetime.datetime(2020, 2, 27) - } - self.dag = DAG('test_dag_id', default_args=args) + cloud_build_hook_constructor.assert_called_once_with( + env={'CONFIG_VAR': 'HELLO'}, + gcs_staging_location='gs://my_bucket/dbt_proj.tar.gz', + project_id='my_project', + gcp_conn_id='test_gcp_conn', + dbt_version='0.19.2', + service_account='dbt-sa@google.com' + ) From 13b25599630abae24a21491ef2c9f27d96ebea05 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 2 Nov 2021 14:41:10 +0100 Subject: [PATCH 37/79] Replace explicit generate_dbt_cli_command with generic one Also take it out of the hook since it is absolutely static. Also since it is not exclusive of the DbtBaseHook nor the DbtCliHook just put it out of the class instead of making it static from within. --- airflow_dbt/hooks/base.py | 353 ++++++++++++++++---------------------- 1 file changed, 146 insertions(+), 207 deletions(-) diff --git a/airflow_dbt/hooks/base.py b/airflow_dbt/hooks/base.py index eeb9f09..803b648 100644 --- a/airflow_dbt/hooks/base.py +++ b/airflow_dbt/hooks/base.py @@ -1,224 +1,163 @@ -from __future__ import print_function - import json from abc import ABC, abstractmethod from typing import Dict, List, Union from airflow.hooks.base_hook import BaseHook +from airflow_dbt.dbt_command_config import DbtCommandConfig -class DbtBaseHook(BaseHook, ABC): - """ - Simple wrapper around the dbt CLI. - :type env: dict - :param env: If set, passed to the dbt executor - :param dbt_bin: The `dbt` CLI. Defaults to `dbt`, so assumes it's on your - `PATH` +def generate_dbt_cli_command( + dbt_bin: str, + command: str, + **params: Union[str, bool], +) -> List[str]: + """ + Creates a CLI string from the keys in the dictionary. If the key is none + it is ignored. If the key is of type boolean the name of the key is added. + If the key is of type string it adds the the key prefixed with tow dashes. + If the key is of type integer it adds the the key prefixed with three + dashes. + dbt_bin and command are mandatory. + Boolean flags must always be positive. + + Available params are: + :param command: The dbt sub-command to run + :type command: str + :param dbt_bin: Path to the dbt binary, defaults to `dbt` assumes it is + available in the PATH. :type dbt_bin: str + :param command: The dbt sub command to run, for example for `dbt run` + the base_command will be `run`. If any other flag not contemplated + must be included it can also be added to this string + :type command: str + :param version: Dbt version to use, in SEMVER. Defaults + to the last one '0.21.0' + :type version: str + :param record_timing_info: Dbt flag to add '--record-timing-info' + :type record_timing_info: bool + :param debug: Dbt flag to add '--debug' + :type debug: bool + :param log_format: Specifies how dbt's logs should be formatted. The + value for this flag can be one of: text, json, or default + :type log_format: str + :param write_json: If set to no it adds the `--no-write-json` Dbt flag + :type write_json: bool + :param strict: Only for use during dbt development. It performs extra + validation of dbt objects and internal consistency checks during + compilation + :type strict: bool + :param warn_error: Converts dbt warnings into errors + :type warn_error: bool + :param partial_parse: configure partial parsing in your project, and + will override the value set in `profiles.yml + :type partial_parse: bool + :param use_experimental_parser: Statically analyze model files in your + project and, if possible, extract needed information 3x faster than + a full Jinja render + :type use_experimental_parser: bool + :param use_colors: Displays colors in dbt logs + :type use_colors: bool + :param profiles_dir: Path to profiles.yaml dir. Can be relative from + the folder the DAG is being run, which usually is the home or de + DAGs folder + :type profiles_dir: str + :param project_dir: Path to the dbt project you want to run. Can be + relative to the path the DAG is being run + :type project_dir: str + :param profile: Which profile to load. Overrides setting in + dbt_project.yml + :type profile: Which profile to load. Overrides setting in + dbt_project.yml + :param target: Which target to load for the given profile + :type target: str + :param config_dir: Sames a profile_dir + :type config_dir: str + :param resource_type: One of: model,snapshot,source,analysis,seed, + exposure,test,default,all + :type resource_type: str + :param vars: Supply variables to the project. This argument overrides + variables defined in your dbt_project.yml file. This argument should + be a YAML string, eg. '{my_variable: my_value}' + :type vars: dict + :param full_refresh: If specified, dbt will drop incremental models and + fully-recalculate the incremental table from the model definition + :type full_refresh: bool + :param data: Run data tests defined in "tests" directory. + :type data: bool + :param schema: Run constraint validations from schema.yml files + :type schema: bool + :param models: Flag used to choose a node or subset of nodes to apply + the command to (v0.210.0 and lower) + :type models: str + :param exclude: Nodes to exclude from the set defined with + select/models + :type exclude: str + :param select: Flag used to choose a node or subset of nodes to apply + the command to (v0.21.0 and higher) + :type select: str + :param selector: Config param to reference complex selects defined in + the config yaml + :type selector: str + :param output: {json,name,path,selector} + :type output: str + :param output_keys: Which keys to output + :type output_keys: str + :param host: Specify the host to listen on for the rpc server + :type host: str + :param port: Specify the port number for the rpc server + :type port: int + :param fail_fast: Stop execution upon a first test failure + :type fail_fast: bool + :param args: + :type args: + :param no_compile: Do not run "dbt compile" as part of docs generation + :type no_compile: bool """ + dbt_command_config_annotations = DbtCommandConfig.__annotations__ + if not dbt_bin or not command: + raise ValueError("dbt_bin and command are mandatory") + command_params = [] + for key, value in params.items(): + # check that the key belongs to DbtCommandConfig keys + if key not in dbt_command_config_annotations.keys(): + raise ValueError(f"{key} is not a valid key") + if value is not None: + # check that the value has the correct type from dbt_command_config_annotations + if type(value) != dbt_command_config_annotations[key]: + raise TypeError(f"{key} has to be of type {dbt_command_config_annotations[key]}") + # if the param is not bool it must have a non null value + cli_param_from_kwarg = "--" + key.replace("_", "-") + command_params.append(cli_param_from_kwarg) + if type(value) is str: + command_params.append(value) + elif type(value) is int: + command_params.append(str(value)) + elif type(value) is dict: + command_params.append(json.dumps(value)) + elif type(value) is bool: + if not value: + raise ValueError( + f"`{key}` cannot be false. Flags will be passed always " + f"afirmatively. If you want to use a negative flag " + f"such as --no-use-colors then provide " + f"`no_use_colors=True`") + return [dbt_bin, command] + command_params - def __init__(self, env: Dict = None): - super().__init__() - self.env = env if env is not None else {} - def generate_dbt_cli_command( - self, - dbt_bin: str = None, - command: str = None, - # global flags - version: bool = False, - record_timing_info: bool = False, - debug: bool = False, - log_format: str = None, # either 'text', 'json' or 'default' - write_json: bool = None, - strict: bool = False, - warn_error: bool = False, - partial_parse: bool = False, - use_experimental_parser: bool = False, - use_colors: bool = None, - # command specific config - profiles_dir: str = None, - project_dir: str = None, - profile: str = None, - target: str = None, - config_dir: str = None, - resource_type: str = None, - vars: Dict = None, - # run specific - full_refresh: bool = False, - # ls specific - data: bool = False, - schema: bool = False, - models: str = None, - exclude: str = None, - select: str = None, - selector: str = None, - output: str = None, - output_keys: str = None, - # rpc specific - host: str = None, - port: str = None, - # test specific - fail_fast: bool = False, - args: dict = None, - no_compile: bool = False, - ) -> List[str]: - """ - Generate the command that will be run based on class properties, - presets and dbt commands +class DbtBaseHook(BaseHook, ABC): + """ + Simple wrapper around the dbt CLI and interface to implement dbt hooks + """ - :param command: The dbt sub-command to run - :type command: str - :param profiles_dir: If set, passed as the `--profiles-dir` argument to - the `dbt` command - :type profiles_dir: str - :param project_dir: If set, passed as the `--project-dir` argument to - the `dbt` command. It is required but by default points to the - current folder: '.' - :type project_dir: str - :param target: If set, passed as the `--target` argument to the `dbt` - command - :type vars: Union[str, dict] - :param vars: If set, passed as the `--vars` argument to the `dbt` - command - :param full_refresh: If `True`, will fully-refresh incremental models. - :type full_refresh: bool - :param data: - :type data: bool - :param schema: - :type schema: bool - :param models: If set, passed as the `--models` argument to the `dbt` - command - :type models: str - :param warn_error: If `True`, treat warnings as errors. - :type warn_error: bool - :param exclude: If set, passed as the `--exclude` argument to the `dbt` - command - :type exclude: str - :param use_colors: If set it adds the flag `--use-colors` or - `--no-use-colors`, depending if True or False. - :param select: If set, passed as the `--select` argument to the `dbt` - command - :type select: str + def __init__(self, env: Dict = None): """ - - dbt_cmd: List[str] = [] - - # if there's no bin do not append it. Rather generate the command - # without the `/path/to/dbt` prefix. That is useful for running it - # inside containers that have already set the entrypoint. - if dbt_bin is not None and not dbt_bin == '': - dbt_cmd.append(dbt_bin) - - # add global flags at the beginning - if version: - dbt_cmd.append('--version') - - if record_timing_info: - dbt_cmd.append('--record-timing-info') - - if debug: - dbt_cmd.append('--debug') - - if log_format is not None: - dbt_cmd.extend(['--log-format', log_format]) - - if write_json is not None: - write_json_flag = '--write-json' if write_json else \ - '--no-write-json' - dbt_cmd.append(write_json_flag) - - if strict: - dbt_cmd.append('--strict') - - if warn_error: - dbt_cmd.append('--warn-error') - - if partial_parse: - dbt_cmd.append('--partial-parse') - - if use_experimental_parser: - dbt_cmd.append('--use-experimental-parser') - - if use_colors is not None: - colors_flag = "--use-colors" if use_colors else "--no-use-colors" - dbt_cmd.append(colors_flag) - - # appends the main command - dbt_cmd.append(command) - - # appends configuration relative to the command - if profiles_dir is not None: - dbt_cmd.extend(['--profiles-dir', profiles_dir]) - - if project_dir is not None: - dbt_cmd.extend(['--project-dir', project_dir]) - - if profile is not None: - dbt_cmd.extend(['--profile', profile]) - - if target is not None: - dbt_cmd.extend(['--target', target]) - - # debug specific - if config_dir is not None: - dbt_cmd.extend(['--config-dir', config_dir]) - - # ls specific - if resource_type is not None: - dbt_cmd.extend(['--resource-type', resource_type]) - - if select is not None: - dbt_cmd.extend(['--select', select]) - - if models is not None: - dbt_cmd.extend(['--models', models]) - - if exclude is not None: - dbt_cmd.extend(['--exclude', exclude]) - - if selector is not None: - dbt_cmd.extend(['--selector', selector]) - - if output is not None: - dbt_cmd.extend(['--output', output]) - - if output_keys is not None: - dbt_cmd.extend(['--output-keys', output_keys]) - - # rpc specific - if host is not None: - dbt_cmd.extend(['--host', host]) - - if port is not None: - dbt_cmd.extend(['--port', str(port)]) - - # run specific - if full_refresh: - dbt_cmd.append('--full-refresh') - - if fail_fast: - dbt_cmd.append('--fail-fast') - - if vars is not None: - dbt_cmd.extend(['--vars', json.dumps(vars)]) - - # run-operation specific - if args is not None: - dbt_cmd.extend(['--args', json.dumps(args)]) - - # test specific - if data: - dbt_cmd.append('--data') - - if schema: - dbt_cmd.append('--schema') - - if no_compile: - dbt_cmd.append('--no-compile') - - return dbt_cmd + :param env: If set will be passed over to cloud build to run in the + dbt step + :type env: dict + """ + super().__init__() + self.env = env if env is not None else {} @abstractmethod def run_dbt(self, dbt_cmd: Union[str, List[str]]): From 0435b58dbf1ee97f7ea5a372744c6a24ac65b27e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 2 Nov 2021 14:41:44 +0100 Subject: [PATCH 38/79] Docs adjustments mainly --- airflow_dbt/hooks/cli.py | 17 +-- airflow_dbt/hooks/google.py | 34 ++--- airflow_dbt/operators/dbt_operator.py | 200 ++++++++++++++++++-------- 3 files changed, 157 insertions(+), 94 deletions(-) diff --git a/airflow_dbt/hooks/cli.py b/airflow_dbt/hooks/cli.py index 6ddc280..a866a34 100644 --- a/airflow_dbt/hooks/cli.py +++ b/airflow_dbt/hooks/cli.py @@ -11,20 +11,15 @@ class DbtCliHook(DbtBaseHook): """ Run the dbt command in the same airflow worker the task is being run. - This requires the `dbt` python package to be installed in it first. Also - the dbt_bin path might not be set in the `PATH` variable, so it could be - necessary to set it in the constructor. - - :type dir: str - :param dir: The directory to run the CLI in - :type env: dict - :param env: If set, passed to the dbt executor - :param dbt_bin: The `dbt` CLI. Defaults to `dbt`, so assumes it's on your - `PATH` - :type dbt_bin: str + This requires the `dbt` python package to be installed in it first. """ def __init__(self, env: Dict = None): + """ + :type env: + :param env: Environment variables that will be passed to the + subprocess. Must be a dictionary of key-values + """ self.sp = SubprocessHook() super().__init__(env=env) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index f5ca8cd..5ff6c8a 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -29,40 +29,27 @@ class DbtCloudBuildHook(DbtBaseHook): """ Runs the dbt command in a Cloud Build job in GCP - :type dir: str - :param dir: Optional, if set the process considers that sources must be - uploaded prior to running the DBT job :type env: dict :param env: If set, passed to the dbt executor - :param dbt_bin: The `dbt` CLI. Defaults to `dbt`, so assumes it's on your - `PATH` - :type dbt_bin: str - :param project_id: GCP Project ID as stated in the console :type project_id: str - :param timeout: Default is set in Cloud Build itself as ten minutes. A - duration in seconds with up to nine fractional digits, terminated by - 's'. Example: "3.5s" - :type timeout: str - :param wait: Waits for the cloud build process to finish. That is waiting - for the DBT command to finish running or run asynchronously - :type wait: bool :param gcp_conn_id: The connection ID to use when fetching connection info. :type gcp_conn_id: str :param gcs_staging_location: Where to store the sources to be fetch later by the cloud build job. It should be the GCS url for a folder. For example: `gs://my-bucket/stored. A sub-folder will be generated to avoid collision between possible different concurrent runs. - :param gcs_staging_location: str + :type gcs_staging_location: str :param dbt_version: the DBT version to be fetched from dockerhub. Defaults to '0.21.0' :type dbt_version: str + :param service_account: email for the service account. If set must be + accompanied by the project_id """ - def __init__( + self, project_id: str = None, - # dir: str = None, gcs_staging_location: str = None, gcp_conn_id: str = "google_cloud_default", dbt_version: str = '0.21.0', @@ -83,7 +70,7 @@ def __init__( self.dbt_version = dbt_version self.cloud_build_hook = CloudBuildHook(gcp_conn_id=gcp_conn_id) self.gcp_conn_id = gcp_conn_id - self.project_id = project_id + self.project_id = self.cloud_build_hook.project_id if project_id is None else project_id self.service_account = service_account super().__init__(env=env) @@ -94,15 +81,15 @@ def get_conn(self) -> Any: def run_dbt(self, dbt_cmd: List[str]): """ - Run the dbt cli + Run the dbt command. In version 5 of the providers :param dbt_cmd: The dbt whole command to run :type dbt_cmd: List[str] """ - """See: https://cloud.google.com/cloud-build/docs/api/reference/rest - /v1/projects.builds""" + # See: https://cloud.google.com/cloud-build/docs/api/reference/rest/v1/projects.builds cloud_build_config = { 'steps': [{ + # use the official dbt docker image from dockerhub 'name': f'fishtownanalytics/dbt:{self.dbt_version}', 'args': dbt_cmd, 'env': [f'{k}={v}' for k, v in self.env.items()] @@ -114,8 +101,11 @@ def run_dbt(self, dbt_cmd: List[str]): } }, 'options': { + # default is legacy and its behaviour is subject to change 'logging': 'GCS_ONLY', }, + # mandatory if using a service_account, it also is relevant as + # transactional data 'logsBucket': self.gcs_staging_bucket, } @@ -149,7 +139,7 @@ def run_dbt(self, dbt_cmd: List[str]): # print result from build logging.info('Build results:\n' + json.dumps(build_results, indent=2)) # set the log_url class param to be read from the "links" - self.log_url = build_results['logUrl'] + return build_results def on_kill(self): """Stopping the build is not implemented until google providers v6""" diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 46e9be0..93f1e61 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -6,60 +6,27 @@ from airflow.utils.decorators import apply_defaults from airflow_dbt.dbt_command_config import DbtCommandConfig +from airflow_dbt.hooks.base import generate_dbt_cli_command from airflow_dbt.hooks.cli import DbtCliHook class DbtBaseOperator(BaseOperator): """ - Base dbt operator - All other dbt operators are derived from this operator. - - :param profiles_dir: If set, passed as the `--profiles-dir` argument to - the `dbt` command - :type profiles_dir: str - :param target: If set, passed as the `--target` argument to the `dbt` - command - :type dir: str - :param dir: The directory to run the CLI in - :param env: If set, passed to the dbt executor - :type env: dict - :type vars: str - :param vars: If set, passed as the `--vars` argument to the `dbt` command - :type vars: dict - :param full_refresh: If `True`, will fully-refresh incremental models. - :type full_refresh: bool - :param models: If set, passed as the `--models` argument to the `dbt` - command - :type models: str - :param warn_error: If `True`, treat warnings as errors. - :type warn_error: bool - :param exclude: If set, passed as the `--exclude` argument to the `dbt` - command - :type exclude: str - :param select: If set, passed as the `--select` argument to the `dbt` - command - :type select: str - :param dbt_bin: The `dbt` CLI. Defaults to `dbt`, so assumes it's on your - `PATH` - :type dbt_bin: str - :param verbose: The operator will log verbosely to the Airflow logs - :type verbose: bool - :param dbt_hook: The dbt hook to use as executor. For now the - implemented ones are: DbtCliHook, DbtCloudBuildHook. It should be an - instance of one of those, or another that inherits from DbtBaseHook. If - not provided by default a DbtCliHook will be instantiated with the - provided params - :type dbt_hook: DbtBaseHook - :param base_command: The dbt sub command to run, for example for `dbt - run` the base_command will be `run`. If any other flag not - contemplated must be included it can also be added to this string - :type base_command: str + Base dbt operator. All other dbt operators should inherit from this one. + + It receives all possible dbt options in the constructor. If no hook is + provided it uses the DbtCliHook to run the generated command. """ ui_color = '#d6522a' - - template_fields = ['env', 'dbt_bin', 'command', 'config'] - + # add all the str/dict params to the templates + dbt_str_params = [ + key for key, value in DbtCommandConfig.__annotations__.items() + if value is str or value is dict + ] + template_fields = ['env', 'dbt_bin', 'command', 'config'] + dbt_str_params + + # noinspection PyShadowingBuiltins @apply_defaults def __init__( self, @@ -72,15 +39,15 @@ def __init__( dir: str = None, # if config was not provided we un-flatten them from the kwargs # global flags - version: bool = False, - record_timing_info: bool = False, - debug: bool = False, + version: bool = None, + record_timing_info: bool = None, + debug: bool = None, log_format: str = None, # either 'text', 'json' or 'default' write_json: bool = None, - strict: bool = False, - warn_error: bool = False, - partial_parse: bool = False, - use_experimental_parser: bool = False, + strict: bool = None, + warn_error: bool = None, + partial_parse: bool = None, + use_experimental_parser: bool = None, use_colors: bool = None, # command specific config profiles_dir: str = None, @@ -91,10 +58,10 @@ def __init__( resource_type: str = None, vars: Dict = None, # run specific - full_refresh: bool = False, + full_refresh: bool = None, # ls specific - data: bool = False, - schema: bool = False, + data: bool = None, + schema: bool = None, models: str = None, exclude: str = None, select: str = None, @@ -105,18 +72,129 @@ def __init__( host: str = None, port: str = None, # test specific - fail_fast: bool = False, + fail_fast: bool = None, args: dict = None, - no_compile=False, + no_compile: bool = None, *vargs, **kwargs ): + """ + :param env: Dictionary with environment variables to be used in the + runtime + :type env: dict + :param dbt_bin: Path to the dbt binary, defaults to `dbt` assumes it is + available in the PATH. + :type dbt_bin: str + :param dbt_hook: The dbt hook to use as executor. For now the + implemented ones are: DbtCliHook, DbtCloudBuildHook. It should be an + instance of one of those, or another that inherits from DbtBaseHook. + If not provided by default a DbtCliHook will be instantiated with + the provided params + :type dbt_hook: DbtBaseHook + :param command: The dbt sub command to run, for example for `dbt run` + the base_command will be `run`. If any other flag not contemplated + must be included it can also be added to this string + :type command: str + :param config: TypedDictionary which accepts all of the commands + related to executing dbt. This way you can separate them from the + ones destined for execution + :type config: DbtCommandConfig + :param dir: Legacy param to set the dbt project directory + :type dir: str + :param version: Dbt version to use, in SEMVER. Defaults + to the last one '0.21.0' + :type version: str + :param record_timing_info: Dbt flag to add '--record-timing-info' + :type record_timing_info: bool + :param debug: Dbt flag to add '--debug' + :type debug: bool + :param log_format: Specifies how dbt's logs should be formatted. The + value for this flag can be one of: text, json, or default + :type log_format: str + :param write_json: If set to no it adds the `--no-write-json` Dbt flag + :type write_json: bool + :param strict: Only for use during dbt development. It performs extra + validation of dbt objects and internal consistency checks during + compilation + :type strict: bool + :param warn_error: Converts dbt warnings into errors + :type warn_error: bool + :param partial_parse: configure partial parsing in your project, and + will override the value set in `profiles.yml + :type partial_parse: bool + :param use_experimental_parser: Statically analyze model files in your + project and, if possible, extract needed information 3x faster than + a full Jinja render + :type use_experimental_parser: bool + :param use_colors: Displays colors in dbt logs + :type use_colors: bool + :param profiles_dir: Path to profiles.yaml dir. Can be relative from + the folder the DAG is being run, which usually is the home or de + DAGs folder + :type profiles_dir: str + :param project_dir: Path to the dbt project you want to run. Can be + relative to the path the DAG is being run + :type project_dir: str + :param profile: Which profile to load. Overrides setting in + dbt_project.yml + :type profile: Which profile to load. Overrides setting in + dbt_project.yml + :param target: Which target to load for the given profile + :type target: str + :param config_dir: Sames a profile_dir + :type config_dir: str + :param resource_type: One of: model,snapshot,source,analysis,seed, + exposure,test,default,all + :type resource_type: str + :param vars: Supply variables to the project. This argument overrides + variables defined in your dbt_project.yml file. This argument should + be a YAML string, eg. '{my_variable: my_value}' + :type vars: dict + :param full_refresh: If specified, dbt will drop incremental models and + fully-recalculate the incremental table from the model definition + :type full_refresh: bool + :param data: Run data tests defined in "tests" directory. + :type data: bool + :param schema: Run constraint validations from schema.yml files + :type schema: bool + :param models: Flag used to choose a node or subset of nodes to apply + the command to (v0.210.0 and lower) + :type models: str + :param exclude: Nodes to exclude from the set defined with + select/models + :type exclude: str + :param select: Flag used to choose a node or subset of nodes to apply + the command to (v0.21.0 and higher) + :type select: str + :param selector: Config param to reference complex selects defined in + the config yaml + :type selector: str + :param output: {json,name,path,selector} + :type output: str + :param output_keys: Which keys to output + :type output_keys: str + :param host: Specify the host to listen on for the rpc server + :type host: str + :param port: Specify the port number for the rpc server + :type port: int + :param fail_fast: Stop execution upon a first test failure + :type fail_fast: bool + :param args: + :type args: + :param no_compile: Do not run "dbt compile" as part of docs generation + :type no_compile: bool + :param vargs: rest of the positional args + :param kwargs: rest of the keyword args + + """ super(DbtBaseOperator, self).__init__(*vargs, **kwargs) if dir is not None: - warnings.warn('"dir" param is deprecated in favor of dbt native ' - 'param "project_dir"') + warnings.warn( + '"dir" param is deprecated in favor of dbt native ' + 'param "project_dir"', PendingDeprecationWarning + ) if project_dir is None: logging.warning('Using "dir" as "project_dir"') project_dir = dir @@ -184,7 +262,7 @@ def execute(self, context: Any): """Runs the provided command in the provided execution environment""" self.instantiate_hook() - dbt_cli_command = self.hook.generate_dbt_cli_command( + dbt_cli_command = generate_dbt_cli_command( dbt_bin=self.dbt_bin, command=self.command, **self.config From e3736f555d6fbd2b667bca37872cb41808df47dc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 2 Nov 2021 14:43:37 +0100 Subject: [PATCH 39/79] Flags are passed True to be reflected in the command line. Also remove verbose which was not part of the dbt cli API --- airflow_dbt/dbt_command_config.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airflow_dbt/dbt_command_config.py b/airflow_dbt/dbt_command_config.py index 038e733..bb87fc4 100644 --- a/airflow_dbt/dbt_command_config.py +++ b/airflow_dbt/dbt_command_config.py @@ -23,6 +23,7 @@ class DbtCommandConfig(TypedDict, total=False): partial_parse: bool use_experimental_parser: bool use_colors: bool + no_use_colors: bool # per command flags profiles_dir: str @@ -31,7 +32,6 @@ class DbtCommandConfig(TypedDict, total=False): vars: dict models: str exclude: str - verbose: bool # run specific full_refresh: bool @@ -45,7 +45,6 @@ class DbtCommandConfig(TypedDict, total=False): # ls specific resource_type: str # models, snapshots, seeds, tests, and sources. - # '--resource-type' select: str models: str exclude: str From 82ec4f40537975e9be09853a1818e22632bf7659 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 13:47:34 +0100 Subject: [PATCH 40/79] Make abstract method implement a single LOC `pass`. thanks @fenimore --- airflow_dbt/hooks/base.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow_dbt/hooks/base.py b/airflow_dbt/hooks/base.py index 803b648..672d2d1 100644 --- a/airflow_dbt/hooks/base.py +++ b/airflow_dbt/hooks/base.py @@ -162,3 +162,4 @@ def __init__(self, env: Dict = None): @abstractmethod def run_dbt(self, dbt_cmd: Union[str, List[str]]): """Run the dbt command""" + pass From b505d597b412e7ee1c2bd0843026c87e2efe9599 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 13:51:52 +0100 Subject: [PATCH 41/79] Explain conditional import of TypedDict as suggested by @andrewrjones --- airflow_dbt/dbt_command_config.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/airflow_dbt/dbt_command_config.py b/airflow_dbt/dbt_command_config.py index bb87fc4..9a51751 100644 --- a/airflow_dbt/dbt_command_config.py +++ b/airflow_dbt/dbt_command_config.py @@ -1,5 +1,7 @@ import sys +# Python versions older than 3.8 have the TypedDict in a different namespace. +# In case we find ourselves in that situation, we use the `older` import if sys.version_info[0] == 3 and sys.version_info[1] >= 8: from typing import TypedDict else: From 239e05f1cc3f1bfc684b54b816b5ef8dc691d593 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 13:57:32 +0100 Subject: [PATCH 42/79] Reuse type in different conditionals, suggested by @johanna-ojeling-datatonic --- airflow_dbt/hooks/base.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/airflow_dbt/hooks/base.py b/airflow_dbt/hooks/base.py index 672d2d1..9ed8d6f 100644 --- a/airflow_dbt/hooks/base.py +++ b/airflow_dbt/hooks/base.py @@ -120,22 +120,23 @@ def generate_dbt_cli_command( command_params = [] for key, value in params.items(): # check that the key belongs to DbtCommandConfig keys - if key not in dbt_command_config_annotations.keys(): + if key not in dbt_command_config_annotations: raise ValueError(f"{key} is not a valid key") if value is not None: + param_value_type = type(value) # check that the value has the correct type from dbt_command_config_annotations - if type(value) != dbt_command_config_annotations[key]: + if param_value_type != dbt_command_config_annotations[key]: raise TypeError(f"{key} has to be of type {dbt_command_config_annotations[key]}") # if the param is not bool it must have a non null value cli_param_from_kwarg = "--" + key.replace("_", "-") command_params.append(cli_param_from_kwarg) - if type(value) is str: + if param_value_type is str: command_params.append(value) - elif type(value) is int: + elif param_value_type is int: command_params.append(str(value)) - elif type(value) is dict: + elif param_value_type is dict: command_params.append(json.dumps(value)) - elif type(value) is bool: + elif param_value_type is bool: if not value: raise ValueError( f"`{key}` cannot be false. Flags will be passed always " From 0cc02050fdf0ba022e7e711c70d968e86cca22a8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 14:55:55 +0100 Subject: [PATCH 43/79] Make project_id optional in the DbtCloudBuildHook MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit A `project_id` must be provided either explicit through the function params or implicit through the provided google hook. Make the first one optional then Co-authored-by: Johanna Öjeling <82930299+johanna-ojeling-datatonic@users.noreply.github.com> --- airflow_dbt/hooks/google.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index 5ff6c8a..dceb9f9 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -49,7 +49,7 @@ class DbtCloudBuildHook(DbtBaseHook): def __init__( self, - project_id: str = None, + project_id: Optional[str] = None, gcs_staging_location: str = None, gcp_conn_id: str = "google_cloud_default", dbt_version: str = '0.21.0', From a1dbbc7880eadd422c527bcd35382c4c5b89a3d4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 14:59:47 +0100 Subject: [PATCH 44/79] Fix typo in pydoc MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Johanna Öjeling <82930299+johanna-ojeling-datatonic@users.noreply.github.com> --- airflow_dbt/hooks/google.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index dceb9f9..9df5462 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -62,7 +62,7 @@ def __init__( if not staging_blob.endswith('.tar.gz'): raise AirflowException( f'The provided blob "{staging_blob}" to a compressed file does not ' + - 'have the right extension ".tar.gz' + 'have the right extension ".tar.gz"' ) self.gcs_staging_bucket = staging_bucket self.gcs_staging_blob = staging_blob From cb0566d4793573c002e3658bb77b2610fca7e2d3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 16:13:10 +0100 Subject: [PATCH 45/79] Make `env` dict for environment variables for google dbt hook optional MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Johanna Öjeling <82930299+johanna-ojeling-datatonic@users.noreply.github.com> --- airflow_dbt/hooks/google.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index 9df5462..85f44d1 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -53,7 +53,7 @@ def __init__( gcs_staging_location: str = None, gcp_conn_id: str = "google_cloud_default", dbt_version: str = '0.21.0', - env: Dict = None, + env: Optional[Dict] = None, service_account=None, ): staging_bucket, staging_blob = _parse_gcs_url(gcs_staging_location) From d4e51846d6bcf8832ab1ef126f5c96001faebd29 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 16:14:45 +0100 Subject: [PATCH 46/79] Make `service_account ` for google dbt hook optional MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Johanna Öjeling <82930299+johanna-ojeling-datatonic@users.noreply.github.com> --- airflow_dbt/hooks/google.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index 85f44d1..a6ac353 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -54,7 +54,7 @@ def __init__( gcp_conn_id: str = "google_cloud_default", dbt_version: str = '0.21.0', env: Optional[Dict] = None, - service_account=None, + service_account: Optional[str] = None, ): staging_bucket, staging_blob = _parse_gcs_url(gcs_staging_location) # we have provided something similar to From 98c90f0a9e557c4e62071ab34245ce228856143e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 16:16:50 +0100 Subject: [PATCH 47/79] Shorten up conditional assignment of project_id MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Johanna Öjeling <82930299+johanna-ojeling-datatonic@users.noreply.github.com> --- airflow_dbt/hooks/google.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index a6ac353..51de4f2 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -70,7 +70,7 @@ def __init__( self.dbt_version = dbt_version self.cloud_build_hook = CloudBuildHook(gcp_conn_id=gcp_conn_id) self.gcp_conn_id = gcp_conn_id - self.project_id = self.cloud_build_hook.project_id if project_id is None else project_id + self.project_id = project_id or self.cloud_build_hook.project_id self.service_account = service_account super().__init__(env=env) From 33af351061223d219c804abfac768f1d1814b86c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 16:17:27 +0100 Subject: [PATCH 48/79] Make `env` dict for environment variables for base dbt hook optional MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Johanna Öjeling <82930299+johanna-ojeling-datatonic@users.noreply.github.com> --- airflow_dbt/hooks/base.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow_dbt/hooks/base.py b/airflow_dbt/hooks/base.py index 9ed8d6f..f96f8fc 100644 --- a/airflow_dbt/hooks/base.py +++ b/airflow_dbt/hooks/base.py @@ -151,7 +151,7 @@ class DbtBaseHook(BaseHook, ABC): Simple wrapper around the dbt CLI and interface to implement dbt hooks """ - def __init__(self, env: Dict = None): + def __init__(self, env: Optional[Dict] = None): """ :param env: If set will be passed over to cloud build to run in the dbt step From 139bfe36b9b851ded2296d510b6cefc03ee7d77d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 16:18:07 +0100 Subject: [PATCH 49/79] Shorten up conditional assignment of env dict MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Johanna Öjeling <82930299+johanna-ojeling-datatonic@users.noreply.github.com> --- airflow_dbt/hooks/base.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow_dbt/hooks/base.py b/airflow_dbt/hooks/base.py index f96f8fc..af6ab7e 100644 --- a/airflow_dbt/hooks/base.py +++ b/airflow_dbt/hooks/base.py @@ -158,7 +158,7 @@ def __init__(self, env: Optional[Dict] = None): :type env: dict """ super().__init__() - self.env = env if env is not None else {} + self.env = env or {} @abstractmethod def run_dbt(self, dbt_cmd: Union[str, List[str]]): From c6b3699f19ee294f6796e4baf189a4fa35da0288 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 16:18:48 +0100 Subject: [PATCH 50/79] Make `env ` for base dbt hook optional MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Johanna Öjeling <82930299+johanna-ojeling-datatonic@users.noreply.github.com> --- airflow_dbt/hooks/cli.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow_dbt/hooks/cli.py b/airflow_dbt/hooks/cli.py index a866a34..c9b4102 100644 --- a/airflow_dbt/hooks/cli.py +++ b/airflow_dbt/hooks/cli.py @@ -14,7 +14,7 @@ class DbtCliHook(DbtBaseHook): This requires the `dbt` python package to be installed in it first. """ - def __init__(self, env: Dict = None): + def __init__(self, env: Optional[Dict] = None): """ :type env: :param env: Environment variables that will be passed to the From face55004b07cc977ea5edab2b055c292bb5ca19 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 16:08:57 +0100 Subject: [PATCH 51/79] Add descriptions to each parametrized dbt test --- tests/hooks/test_dbt_hook.py | 167 ++++++++++++++++++++++++++--------- 1 file changed, 126 insertions(+), 41 deletions(-) diff --git a/tests/hooks/test_dbt_hook.py b/tests/hooks/test_dbt_hook.py index 0c73d75..a769b8b 100644 --- a/tests/hooks/test_dbt_hook.py +++ b/tests/hooks/test_dbt_hook.py @@ -12,23 +12,36 @@ from airflow_dbt.hooks.cli import DbtCliHook from airflow_dbt.hooks.google import DbtCloudBuildHook - -@mark.parametrize( - ["dbt_bin", "command", "params", "expected_command"], [ - ("dbt", "run", {}, ["dbt", "run"]), +cli_command_from_params_data = [ + [("dbt", "run", {}, ["dbt", "run"]), "regular dbt run"], # check it runs with empty params - ("dbt", None, {}, ValueError()), # check it fails with no command - (None, "run", {}, ValueError()), # check it fails with no dbt_bin - ("dbt", "test", {}, ["dbt", "test"]), # test without params - ("dbt", "test", {'non_existing_param'}, TypeError()), + [("dbt", None, {}, ValueError()), "it fails with no command"], + [(None, "run", {}, ValueError()), "it fails with no dbt_bin"], + [("dbt", "test", {}, ["dbt", "test"]), "test without params"], + [ + ("dbt", "test", {'non_existing_param'}, TypeError()), + "invalid param raises TypeError" + ], # test invalid param - ("dbt", "test", {'--models': None}, ValueError()), + [ + ("dbt", "test", {'--models': None}, ValueError()), + "required --models value raises ValueError if not provided" + ], # test mandatory value - ("dbt", "test", {'--models': 3}, ValueError()), # test wrong type - ("/bin/dbt", "test", {}, ["/bin/dbt", "test"]), # test dbt path - ("dbt", "run", {'full_refresh': False}, ValueError()), + [ + ("dbt", "test", {'--models': 3}, ValueError()), + "required --models value raises ValueError if not correct type" + ], + [ + ("/bin/dbt", "test", {}, ["/bin/dbt", "test"]), + "dbt_bin other than the default gets passed through" + ], + [ + ("dbt", "run", {'full_refresh': False}, ValueError()), + "flags param fails if contains False value" + ], # test flags always positive - ('/home/airflow/.local/bin/dbt', 'run', { + [('/home/airflow/.local/bin/dbt', 'run', { 'full_refresh': True, 'profiles_dir': '/opt/airflow/dags/dbt_project', 'project_dir': '/opt/airflow/dags/project_dir', @@ -39,65 +52,137 @@ '--project-dir', '/opt/airflow/dags/project_dir', '--vars', '{"execution_date": "2021-01-01"}', '--select', 'my_model']), + "fully fledged dbt run with all types of params" + ], # test all the params - ("dbt", "test", {'profiles_dir': '/path/profiles_folder'}, + [ + ("dbt", "test", {'profiles_dir': '/path/profiles_folder'}, ["dbt", "test", "--profiles-dir", "/path/profiles_folder"]), - ("dbt", "run", {'project_dir': '/path/dbt_project_dir'}, + "test profiles_dir param" + ], + [ + ("dbt", "run", {'project_dir': '/path/dbt_project_dir'}, ["dbt", "run", "--project-dir", "/path/dbt_project_dir"]), - ("dbt", "test", {'target': 'model_target'}, + "test project_dir param" + ], + [ + ("dbt", "test", {'target': 'model_target'}, ["dbt", "test", "--target", "model_target"]), - ("dbt", "test", {'vars': {"hello": "world"}}, + "test target param" + ], + [ + ("dbt", "test", {'vars': {"hello": "world"}}, ["dbt", "test", "--vars", '{"hello": "world"}']), - ("dbt", "run", {'models': 'my_model'}, + "test vars param" + ], + [ + ("dbt", "run", {'models': 'my_model'}, ["dbt", "run", "--models", "my_model"]), - ("dbt", "run", {'exclude': 'my_model'}, - ["dbt", "run", "--exclude", "my_model"]), - ("dbt", "run", {'exclude': 'my_model'}, + "test models param" + ], + [ + ("dbt", "run", {'exclude': 'my_model'}, ["dbt", "run", "--exclude", "my_model"]), + "test exclude param" + ], # run specific params - ("dbt", "run", {'full_refresh': True}, + [ + ("dbt", "run", {'full_refresh': True}, ["dbt", "run", "--full-refresh"]), - ("dbt", "run", {'full_refresh': 3}, TypeError()), - ("dbt", "run", {'full_refresh': 'hello'}, TypeError()), - ("dbt", "run", {'profile': 'test_profile'}, + "[dbt run] test full_refresh flag succeeds" + ], + [ + ("dbt", "run", {'full_refresh': 3}, TypeError()), + "[dbt run] test full_refresh param fails if not bool but integer" + ], + [ + ("dbt", "run", {'full_refresh': 'hello'}, TypeError()), + "[dbt run] test full_refresh project_dir fails if not bool but string" + ], + [ + ("dbt", "run", {'profile': 'test_profile'}, ["dbt", "run", "--profile", "test_profile"]), + "[dbt run] test profile param" + ], # docs specific params - ("dbt", "docs", {'no_compile': True}, + [ + ("dbt", "docs", {'no_compile': True}, ["dbt", "docs", "--no-compile"]), - + "test no_compile flag succeeds" + ], # debug specific params - ("dbt", "debug", {'config_dir': '/path/to/config_dir'}, + [ + ("dbt", "debug", {'config_dir': '/path/to/config_dir'}, ["dbt", "debug", "--config-dir", '/path/to/config_dir']), + "[dbt debug] test config_dir param" + ], # ls specific params - ("dbt", "ls", {'resource_type': '/path/to/config_dir'}, + [ + ("dbt", "ls", {'resource_type': '/path/to/config_dir'}, ["dbt", "ls", "--resource-type", '/path/to/config_dir']), - ("dbt", "ls", {'select': 'my_model'}, + "[dbt ls] test resource_type param" + ], + [ + ("dbt", "ls", {'select': 'my_model'}, ["dbt", "ls", "--select", "my_model"]), - ("dbt", "ls", {'exclude': 'my_model'}, + "[dbt ls] test select param" + ], + [ + ("dbt", "ls", {'exclude': 'my_model'}, ["dbt", "ls", "--exclude", "my_model"]), - ("dbt", "ls", {'output': 'my_model'}, + "[dbt ls] test exclude param" + ], + [ + ("dbt", "ls", {'output': 'my_model'}, ["dbt", "ls", "--output", "my_model"]), - ("dbt", "ls", {'output_keys': 'my_model'}, + "[dbt ls] test output param" + ], + [ + ("dbt", "ls", {'output_keys': 'my_model'}, ["dbt", "ls", "--output-keys", "my_model"]), + "[dbt ls] test output_keys param" + ], # rpc specific params - ("dbt", "rpc", {'host': 'http://my-host-url.com'}, + [ + ("dbt", "rpc", {'host': 'http://my-host-url.com'}, ["dbt", "rpc", "--host", 'http://my-host-url.com']), - ("dbt", "rpc", {'port': '8080'}, TypeError()), - ("dbt", "rpc", {'port': 8080}, ["dbt", "rpc", "--port", '8080']), + "[dbt rpc] test host param" + ], + [ + ("dbt", "rpc", {'port': '8080'}, TypeError()), + "[dbt rpc] test port param fails if not integer" + ], + [ + ("dbt", "rpc", {'port': 8080}, ["dbt", "rpc", "--port", '8080']), + "[dbt rpc] test port param" + ], # run specific params - ("dbt", "run", {'fail_fast': True}, ["dbt", "run", "--fail-fast"]), + [ + ("dbt", "run", {'fail_fast': True}, ["dbt", "run", "--fail-fast"]), + "[dbt run] test fail_fast flag succeeds" + ], # test specific params - ("dbt", "test", {'data': True}, ["dbt", "test", '--data']), - ("dbt", "test", {'schema': True}, ["dbt", "test", '--schema']), - # without params - + [ + ("dbt", "test", {'data': True}, ["dbt", "test", '--data']), + "[dbt test] test data flag succeeds" + ], + [ + ("dbt", "test", {'schema': True}, ["dbt", "test", '--schema']), + "[dbt test] test schema flag succeeds" + ], ] + + +@mark.parametrize( + ["dbt_bin", "command", "params", "expected_command"], + [test_params[0] for test_params in cli_command_from_params_data], + ids=[test_params[1] for test_params in cli_command_from_params_data] ) def test_create_cli_command_from_params( dbt_bin: str, From 6f7a507f428c750ccb43ece60da575c6fc108060 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 16:21:52 +0100 Subject: [PATCH 52/79] Add README.md code examples --- README.md | 68 +++++++++++++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 66 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 9a4368d..df197aa 100644 --- a/README.md +++ b/README.md @@ -67,7 +67,7 @@ There are five operators currently implemented: * Calls [`dbt test`](https://docs.getdbt.com/docs/test) -Each of the above operators accept the following arguments: +Each of the above operators accept the arguments in [here (dbt_command_config)](airflow_dbt/dbt_command_config.py). The main ones being: * `profiles_dir` * If set, passed as the `--profiles-dir` argument to the `dbt` command @@ -96,6 +96,68 @@ Typically you will want to use the `DbtRunOperator`, followed by the `DbtTestOpe You can also use the hook directly. Typically this can be used for when you need to combine the `dbt` command with another task in the same operators, for example running `dbt docs` and uploading the docs to somewhere they can be served from. +## A more advanced example: + +If want to run your `dbt` project other tan in the airflow worker you can use +the `DbtCloudBuildHook` and apply it to the `DbtBaseOperator` or simply use the +provided `DbtCloudBuildOperator`: + +```python +from airflow_dbt.hooks import DbtCloudBuildHook +from airflow_dbt.operators import DbtBaseOperator, DbtCloudBuildOperator +DbtBaseOperator( + task_id='provide_hook', + command='run', + use_colors=False, + config={ + 'profiles_dir': './jaffle-shop', + 'project_dir': './jaffle-shop', + }, + dbt_hook=DbtCloudBuildHook( + gcs_staging_location='gs://my-bucket/compressed-dbt-project.tar.gz' + ) +) + +DbtCloudBuildOperator( + task_id='default_hook_cloudbuild', + gcs_staging_location='gs://my-bucket/compressed-dbt-project.tar.gz', + command='run', + use_colors=False, + config={ + 'profiles_dir': './jaffle-shop', + 'project_dir': './jaffle-shop', + }, +) +``` + +You can either define the dbt params/config/flags in the operator or you can +group them into a `config` param. They both have validation, but only the config +has templating. The following two tasks are equivalent: + +```python +from airflow_dbt.operators.dbt_operator import DbtBaseOperator + +DbtBaseOperator( + task_id='config_param', + command='run', + config={ + 'profiles_dir': './jaffle-shop', + 'project_dir': './jaffle-shop', + 'dbt_bin': '/usr/local/airflow/.local/bin/dbt', + 'use_colors': False + } +) + +DbtBaseOperator( + task_id='flat_config', + command='run', + profiles_dir='./jaffle-shop', + project_dir='./jaffle-shop', + dbt_bin='/usr/local/airflow/.local/bin/dbt', + use_colors=False +) +``` + ## Building Locally To install from the repository: @@ -147,7 +209,9 @@ If you use MWAA, you just need to update the `requirements.txt` file and add `ai Then you can have your dbt code inside a folder `{DBT_FOLDER}` in the dags folder on S3 and configure the dbt task like below: ```python -dbt_run = DbtRunOperator( +from airflow_dbt.operators.dbt_operator import DbtRunOperator + +dbt_run=DbtRunOperator( task_id='dbt_run', dbt_bin='/usr/local/airflow/.local/bin/dbt', profiles_dir='/usr/local/airflow/dags/{DBT_FOLDER}/', From 1c41b361c99bfffdda34619033b25f68a1d7c129 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 16:53:45 +0100 Subject: [PATCH 53/79] Split check values in two for readability --- airflow_dbt/hooks/base.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/airflow_dbt/hooks/base.py b/airflow_dbt/hooks/base.py index af6ab7e..e905fd6 100644 --- a/airflow_dbt/hooks/base.py +++ b/airflow_dbt/hooks/base.py @@ -115,8 +115,10 @@ def generate_dbt_cli_command( :type no_compile: bool """ dbt_command_config_annotations = DbtCommandConfig.__annotations__ - if not dbt_bin or not command: - raise ValueError("dbt_bin and command are mandatory") + if not dbt_bin: + raise ValueError("dbt_bin is mandatory") + if not command: + raise ValueError("command mandatory") command_params = [] for key, value in params.items(): # check that the key belongs to DbtCommandConfig keys From 52884ae3097713b3fa8e2c01bb0c440939cf4285 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 17:24:28 +0100 Subject: [PATCH 54/79] Extract provider version check to a function and test it, also typos --- airflow_dbt/hooks/base.py | 3 +-- airflow_dbt/hooks/cli.py | 2 +- airflow_dbt/hooks/google.py | 47 +++++++++++++++++++++++++----------- tests/hooks/test_dbt_hook.py | 37 +++++++++++++++++++++++++++- 4 files changed, 71 insertions(+), 18 deletions(-) diff --git a/airflow_dbt/hooks/base.py b/airflow_dbt/hooks/base.py index e905fd6..45fdbb9 100644 --- a/airflow_dbt/hooks/base.py +++ b/airflow_dbt/hooks/base.py @@ -1,6 +1,6 @@ import json from abc import ABC, abstractmethod -from typing import Dict, List, Union +from typing import Dict, List, Optional, Union from airflow.hooks.base_hook import BaseHook @@ -121,7 +121,6 @@ def generate_dbt_cli_command( raise ValueError("command mandatory") command_params = [] for key, value in params.items(): - # check that the key belongs to DbtCommandConfig keys if key not in dbt_command_config_annotations: raise ValueError(f"{key} is not a valid key") if value is not None: diff --git a/airflow_dbt/hooks/cli.py b/airflow_dbt/hooks/cli.py index c9b4102..93c5ce6 100644 --- a/airflow_dbt/hooks/cli.py +++ b/airflow_dbt/hooks/cli.py @@ -1,6 +1,6 @@ from __future__ import print_function -from typing import Any, Dict, List, Union +from typing import Any, Dict, List, Optional, Union from airflow import AirflowException from airflow.hooks.subprocess import SubprocessHook diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index 51de4f2..90d9f91 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -1,5 +1,5 @@ import logging -from typing import Any, Dict, List +from typing import Any, Dict, List, Optional from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.cloud_build import CloudBuildHook @@ -10,19 +10,38 @@ from airflow_dbt.hooks.base import DbtBaseHook -# Check we're using the right google provider version. As composer is the -# most brad used Airflow installation we will default to the latest version -# composer is using -google_providers_version = get_provider_info().get('versions')[0] -v_min = version.parse('5.0.0') -v_max = version.parse('6.0.0') -v_provider = version.parse(google_providers_version) -if not v_min <= v_provider < v_max: - raise Exception( - 'The provider "apache-airflow-providers-google" version "' - f'{google_providers_version}" is not compatible with the current API. ' - f'Please install a compatible version in the range [{v_min}, {v_max})"' - ) +MIN_AIRFLOW_GOOGLE_PROVIDER_VERSION = '5.0.0' +MAX_AIRFLOW_GOOGLE_PROVIDER_VERSION = '6.0.0' + + +def check_google_provider_version(version_min: str, version_max: str) -> None: + """ + Check we're using the right Google provider version. As Cloud Composer is + the most broadly used Airflow installation we will default to the latest + version composer is using + + :param version_min: Minimum version of the Google provider in semver format + :type version_min: str + :param version_max: Maximum version of the Google provider in semver format + :type version_max: str + """ + google_providers_version = get_provider_info().get('versions')[0] + version_min = version.parse(version_min) + version_max = version.parse(version_max) + version_provider = version.parse(google_providers_version) + if not version_min <= version_provider < version_max: + raise Exception( + 'The provider "apache-airflow-providers-google" version "' + f'{google_providers_version}" is not compatible with the current ' + 'API. Please install a compatible version in the range ' + f'>={version_min}, <{version_max}"' + ) + + +check_google_provider_version( + version_min=MIN_AIRFLOW_GOOGLE_PROVIDER_VERSION, + version_max=MAX_AIRFLOW_GOOGLE_PROVIDER_VERSION, +) class DbtCloudBuildHook(DbtBaseHook): diff --git a/tests/hooks/test_dbt_hook.py b/tests/hooks/test_dbt_hook.py index a769b8b..400ef10 100644 --- a/tests/hooks/test_dbt_hook.py +++ b/tests/hooks/test_dbt_hook.py @@ -10,7 +10,10 @@ from airflow_dbt.dbt_command_config import DbtCommandConfig from airflow_dbt.hooks.base import generate_dbt_cli_command from airflow_dbt.hooks.cli import DbtCliHook -from airflow_dbt.hooks.google import DbtCloudBuildHook +from airflow_dbt.hooks.google import ( + DbtCloudBuildHook, + check_google_provider_version, +) cli_command_from_params_data = [ [("dbt", "run", {}, ["dbt", "run"]), "regular dbt run"], @@ -294,3 +297,35 @@ def test_create_build(self, _, MockCloudBuildHook): body=expected_body, project_id='test_project_id' ) + + +@pytest.mark.parametrize( + ['min_version', 'max_version', 'versions', 'expected_result'], + [ + ('5.0.0', '6.0.0', ['5.0.0', '4.0.0'], None), + ('5.0.0', '6.0.0', ['4.0.0', '3.0.0'], Exception), + ('5.0.0', '6.0.0', ['6.0.0', '5.0.0'], Exception), + ], + ids=[ + 'provider version within min and max allowed versions', + 'provider version below min allowed versions', + 'provider version above max allowed versions', + ] +) +@patch('airflow_dbt.hooks.google.get_provider_info') +def test_check_google_provider_version( + mock_get_provider_info, + min_version, + max_version, + versions, + expected_result +): + mock_get_provider_info.return_value = {'versions': versions} + if expected_result is None: + check_google_provider_version( + min_version, + max_version + ) + else: + with pytest.raises(expected_result): + check_google_provider_version(min_version, max_version) From df62625e78e53fc4d6a8984c9149d62c860ee468 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 7 Dec 2021 18:27:08 +0100 Subject: [PATCH 55/79] Fix conditionals comparing with None since None is `falsy` in python --- airflow_dbt/hooks/google.py | 6 +++--- airflow_dbt/operators/google.py | 8 ++++---- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index 90d9f91..29b2bbc 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -128,10 +128,10 @@ def run_dbt(self, dbt_cmd: List[str]): 'logsBucket': self.gcs_staging_bucket, } - if self.service_account is not None: - sa_path = f'projects/{self.project_id}/serviceAccounts/' \ + if self.service_account: + service_account_path_path = f'projects/{self.project_id}/serviceAccounts/' \ f'{self.service_account}' - cloud_build_config['serviceAccount'] = sa_path + cloud_build_config['serviceAccount'] = service_account_path_path cloud_build_config_str = json.dumps(cloud_build_config, indent=2) logging.info( diff --git a/airflow_dbt/operators/google.py b/airflow_dbt/operators/google.py index add8f32..38e087c 100644 --- a/airflow_dbt/operators/google.py +++ b/airflow_dbt/operators/google.py @@ -72,13 +72,13 @@ def instantiate_hook(self): 'env': self.env, 'gcs_staging_location': self.gcs_staging_location, } - if self.project_id is not None: + if self.project_id: hook_config['project_id'] = self.project_id - if self.gcp_conn_id is not None: + if self.gcp_conn_id: hook_config['gcp_conn_id'] = self.gcp_conn_id - if self.dbt_version is not None: + if self.dbt_version: hook_config['dbt_version'] = self.dbt_version - if self.service_account is not None: + if self.service_account: hook_config['service_account'] = self.service_account self.hook = DbtCloudBuildHook(**hook_config) From 642f6dd5edf9eac83161d63dcf85f096cb18bd4a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Thu, 13 Jan 2022 18:38:11 +0100 Subject: [PATCH 56/79] Fix python files not being detected in setup by `find_packages`: The setup feature `find_packages` detects only packages containing the `__init__.py` file. As we removed such to unify the imports when installing those python files where not copied. --- airflow_dbt/__init__.py | 0 airflow_dbt/hooks/__init__.py | 0 airflow_dbt/operators/__init__.py | 0 3 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 airflow_dbt/__init__.py create mode 100644 airflow_dbt/hooks/__init__.py create mode 100644 airflow_dbt/operators/__init__.py diff --git a/airflow_dbt/__init__.py b/airflow_dbt/__init__.py new file mode 100644 index 0000000..e69de29 diff --git a/airflow_dbt/hooks/__init__.py b/airflow_dbt/hooks/__init__.py new file mode 100644 index 0000000..e69de29 diff --git a/airflow_dbt/operators/__init__.py b/airflow_dbt/operators/__init__.py new file mode 100644 index 0000000..e69de29 From 614309bd627b9f57d88b8d57a659b4f32c58e411 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Thu, 13 Jan 2022 18:38:40 +0100 Subject: [PATCH 57/79] Bump version --- airflow_dbt/__version__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow_dbt/__version__.py b/airflow_dbt/__version__.py index 9f59e6c..e2b2363 100644 --- a/airflow_dbt/__version__.py +++ b/airflow_dbt/__version__.py @@ -1,3 +1,3 @@ -VERSION = (0, 4, 0) +VERSION = (0, 5, 0) __version__ = '.'.join(map(str, VERSION)) From 23e16a6cb4c024c7f8c422ce82942e6bbaab2850 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Thu, 13 Jan 2022 19:13:53 +0100 Subject: [PATCH 58/79] Make operator font color white, easier to read against a mid-dark bg --- airflow_dbt/operators/dbt_operator.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 93f1e61..0b6b20e 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -19,6 +19,7 @@ class DbtBaseOperator(BaseOperator): """ ui_color = '#d6522a' + ui_fgcolor = "white" # add all the str/dict params to the templates dbt_str_params = [ key for key, value in DbtCommandConfig.__annotations__.items() From ce219a06fa6a304184ad37eb28fa8f03b4a33e30 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 14 Jan 2022 03:45:12 +0100 Subject: [PATCH 59/79] Allow false flags and reorder dbt render --- airflow_dbt/hooks/base.py | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/airflow_dbt/hooks/base.py b/airflow_dbt/hooks/base.py index 45fdbb9..a6c8859 100644 --- a/airflow_dbt/hooks/base.py +++ b/airflow_dbt/hooks/base.py @@ -129,7 +129,10 @@ def generate_dbt_cli_command( if param_value_type != dbt_command_config_annotations[key]: raise TypeError(f"{key} has to be of type {dbt_command_config_annotations[key]}") # if the param is not bool it must have a non null value - cli_param_from_kwarg = "--" + key.replace("_", "-") + flag_prefix = '' + if param_value_type is bool and not value: + flag_prefix = 'no-' + cli_param_from_kwarg = "--" + flag_prefix + key.replace("_", "-") command_params.append(cli_param_from_kwarg) if param_value_type is str: command_params.append(value) @@ -137,14 +140,8 @@ def generate_dbt_cli_command( command_params.append(str(value)) elif param_value_type is dict: command_params.append(json.dumps(value)) - elif param_value_type is bool: - if not value: - raise ValueError( - f"`{key}` cannot be false. Flags will be passed always " - f"afirmatively. If you want to use a negative flag " - f"such as --no-use-colors then provide " - f"`no_use_colors=True`") - return [dbt_bin, command] + command_params + + return [dbt_bin, *command_params, command] class DbtBaseHook(BaseHook, ABC): From 22ddab129eb02582eb58ecea9b0fced4ca267982 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 14 Jan 2022 03:47:41 +0100 Subject: [PATCH 60/79] Share DbtBaseOperator props between subclasses --- airflow_dbt/operators/dbt_operator.py | 56 +++++++++++++++++---------- airflow_dbt/operators/google.py | 10 +---- 2 files changed, 38 insertions(+), 28 deletions(-) diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 0b6b20e..496fdd2 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -21,13 +21,20 @@ class DbtBaseOperator(BaseOperator): ui_color = '#d6522a' ui_fgcolor = "white" # add all the str/dict params to the templates - dbt_str_params = [ - key for key, value in DbtCommandConfig.__annotations__.items() - if value is str or value is dict - ] - template_fields = ['env', 'dbt_bin', 'command', 'config'] + dbt_str_params - - # noinspection PyShadowingBuiltins + template_fields = ['dbt_env', 'dbt_bin', 'dbt_command', 'dbt_config'] + template_fields_renderers = { + 'dbt_env': 'json', + 'dbt_config': 'json', + } + + dbt_env: Dict + dbt_bin: str + dbt_command: str + dbt_config: Dict + dbt_hook: DbtBaseHook + dbt_cli_command: List[str] + + # noinspection PyShadowingBuiltins, PyDeprecation @apply_defaults def __init__( self, @@ -200,10 +207,13 @@ def __init__( logging.warning('Using "dir" as "project_dir"') project_dir = dir - self.env = {} if env is None else env + self.dbt_env = env or {} self.dbt_bin = dbt_bin - self.command = command - self.config = config if config is not None else { + self.dbt_command = command + # defaults to an empty dict + config = config or {} + # overrides with the top level config + config.update({ # global flags 'version': version, 'record_timing_info': record_timing_info, @@ -245,30 +255,36 @@ def __init__( # test specific 'data': data, 'schema': schema, + }) + # filter out None values from the constructor + config = { + key: val + for key, val in config.items() + if val is not None } - self.env = env - self.hook = dbt_hook + self.dbt_config = config + self.dbt_env = env + self.dbt_hook = dbt_hook def instantiate_hook(self): """ Instantiates the underlying dbt hook. This has to be deferred until after the constructor or the templated params wont be interpolated. """ - dbt_hook = self.hook - self.hook = dbt_hook if dbt_hook is not None else DbtCliHook( - env=self.env, + dbt_hook = self.dbt_hook + self.dbt_hook = dbt_hook if dbt_hook is not None else DbtCliHook( + env=self.dbt_env, ) def execute(self, context: Any): """Runs the provided command in the provided execution environment""" self.instantiate_hook() - - dbt_cli_command = generate_dbt_cli_command( + self.dbt_cli_command = generate_dbt_cli_command( dbt_bin=self.dbt_bin, - command=self.command, - **self.config + command=self.dbt_command, + **self.dbt_config ) - self.hook.run_dbt(dbt_cli_command) + self.dbt_hook.run_dbt(self.dbt_cli_command) class DbtRunOperator(DbtBaseOperator): diff --git a/airflow_dbt/operators/google.py b/airflow_dbt/operators/google.py index 38e087c..2e2e17b 100644 --- a/airflow_dbt/operators/google.py +++ b/airflow_dbt/operators/google.py @@ -38,12 +38,9 @@ class DbtCloudBuildOperator(DbtBaseOperator): def __init__( self, gcs_staging_location: str, - env: Dict = None, - config: DbtCommandConfig = None, project_id: str = None, gcp_conn_id: str = None, dbt_version: str = None, - dbt_bin: Optional[str] = None, service_account: str = None, *args, **kwargs @@ -55,9 +52,6 @@ def __init__( self.service_account = service_account super(DbtCloudBuildOperator, self).__init__( - env=env, - config=config, - dbt_bin=dbt_bin, *args, **kwargs ) @@ -69,7 +63,7 @@ def instantiate_hook(self): not been yet interpolated. """ hook_config = { - 'env': self.env, + 'env': self.dbt_env, 'gcs_staging_location': self.gcs_staging_location, } if self.project_id: @@ -81,4 +75,4 @@ def instantiate_hook(self): if self.service_account: hook_config['service_account'] = self.service_account - self.hook = DbtCloudBuildHook(**hook_config) + self.dbt_hook = DbtCloudBuildHook(**hook_config) From de1fec42fc8cd6fe131a61b49a2713d9a68c5ba8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 14 Jan 2022 03:48:14 +0100 Subject: [PATCH 61/79] Ignore deprecated @apply_defaults, It's needed for backwards compat --- airflow_dbt/operators/dbt_operator.py | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 496fdd2..86bdbc5 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -1,12 +1,13 @@ import logging import warnings -from typing import Any, Dict, Optional +from typing import Any, Dict, List, Optional from airflow.models import BaseOperator +# noinspection PyDeprecation from airflow.utils.decorators import apply_defaults from airflow_dbt.dbt_command_config import DbtCommandConfig -from airflow_dbt.hooks.base import generate_dbt_cli_command +from airflow_dbt.hooks.base import DbtBaseHook, generate_dbt_cli_command from airflow_dbt.hooks.cli import DbtCliHook @@ -269,7 +270,7 @@ def __init__( def instantiate_hook(self): """ Instantiates the underlying dbt hook. This has to be deferred until - after the constructor or the templated params wont be interpolated. + after the constructor or the templated params won't be interpolated. """ dbt_hook = self.dbt_hook self.dbt_hook = dbt_hook if dbt_hook is not None else DbtCliHook( @@ -290,6 +291,7 @@ def execute(self, context: Any): class DbtRunOperator(DbtBaseOperator): """Runs a dbt run command""" + # noinspection PyDeprecation @apply_defaults def __init__(self, *args, **kwargs): super().__init__(*args, command='run', **kwargs) @@ -298,6 +300,7 @@ def __init__(self, *args, **kwargs): class DbtTestOperator(DbtBaseOperator): """Runs a dbt test command""" + # noinspection PyDeprecation @apply_defaults def __init__(self, *args, **kwargs): super().__init__(*args, command='test', **kwargs) @@ -306,6 +309,7 @@ def __init__(self, *args, **kwargs): class DbtDocsGenerateOperator(DbtBaseOperator): """Runs a dbt docs generate command""" + # noinspection PyDeprecation @apply_defaults def __init__(self, *args, **kwargs): super().__init__(*args, command='docs generate', **kwargs) @@ -314,6 +318,7 @@ def __init__(self, *args, **kwargs): class DbtSnapshotOperator(DbtBaseOperator): """Runs a dbt snapshot command""" + # noinspection PyDeprecation @apply_defaults def __init__(self, *args, **kwargs): super().__init__(*args, command='snapshot', **kwargs) @@ -322,6 +327,7 @@ def __init__(self, *args, **kwargs): class DbtSeedOperator(DbtBaseOperator): """Runs a dbt seed command""" + # noinspection PyDeprecation @apply_defaults def __init__(self, *args, **kwargs): super().__init__(*args, command='seed', **kwargs) @@ -330,6 +336,7 @@ def __init__(self, *args, **kwargs): class DbtDepsOperator(DbtBaseOperator): """Runs a dbt deps command""" + # noinspection PyDeprecation @apply_defaults def __init__(self, *args, **kwargs): super().__init__(*args, command='deps', **kwargs) @@ -338,6 +345,7 @@ def __init__(self, *args, **kwargs): class DbtCleanOperator(DbtBaseOperator): """Runs a dbt clean command""" + # noinspection PyDeprecation @apply_defaults def __init__(self, *args, **kwargs): super().__init__(*args, command='clean', **kwargs) From 0f6e313eade1a1b8136f3f9aa1ef6b1ebd340557 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 14 Jan 2022 04:17:41 +0100 Subject: [PATCH 62/79] Fix dbt config, some params for dbt and some for the specific command --- airflow_dbt/hooks/base.py | 58 +++++++++++++++------------ airflow_dbt/operators/dbt_operator.py | 20 ++++++++- 2 files changed, 52 insertions(+), 26 deletions(-) diff --git a/airflow_dbt/hooks/base.py b/airflow_dbt/hooks/base.py index a6c8859..747af77 100644 --- a/airflow_dbt/hooks/base.py +++ b/airflow_dbt/hooks/base.py @@ -2,15 +2,44 @@ from abc import ABC, abstractmethod from typing import Dict, List, Optional, Union +# noinspection PyDeprecation from airflow.hooks.base_hook import BaseHook from airflow_dbt.dbt_command_config import DbtCommandConfig +def render_config(config: dict[str, Union[str,bool]]) -> List[str]: + """Renders a dictionary of options into a list of cli strings""" + dbt_command_config_annotations = DbtCommandConfig.__annotations__ + command_params = [] + for key, value in config.items(): + if key not in dbt_command_config_annotations: + raise ValueError(f"{key} is not a valid key") + if value is not None: + param_value_type = type(value) + # check that the value has the correct type from dbt_command_config_annotations + if param_value_type != dbt_command_config_annotations[key]: + raise TypeError(f"{key} has to be of type {dbt_command_config_annotations[key]}") + # if the param is not bool it must have a non null value + flag_prefix = '' + if param_value_type is bool and not value: + flag_prefix = 'no-' + cli_param_from_kwarg = "--" + flag_prefix + key.replace("_", "-") + command_params.append(cli_param_from_kwarg) + if param_value_type is str: + command_params.append(value) + elif param_value_type is int: + command_params.append(str(value)) + elif param_value_type is dict: + command_params.append(json.dumps(value)) + return command_params + + def generate_dbt_cli_command( dbt_bin: str, command: str, - **params: Union[str, bool], + base_config: Dict[str, Union[str, bool]], + command_config: Dict[str, Union[str, bool]], ) -> List[str]: """ Creates a CLI string from the keys in the dictionary. If the key is none @@ -114,34 +143,13 @@ def generate_dbt_cli_command( :param no_compile: Do not run "dbt compile" as part of docs generation :type no_compile: bool """ - dbt_command_config_annotations = DbtCommandConfig.__annotations__ if not dbt_bin: raise ValueError("dbt_bin is mandatory") if not command: raise ValueError("command mandatory") - command_params = [] - for key, value in params.items(): - if key not in dbt_command_config_annotations: - raise ValueError(f"{key} is not a valid key") - if value is not None: - param_value_type = type(value) - # check that the value has the correct type from dbt_command_config_annotations - if param_value_type != dbt_command_config_annotations[key]: - raise TypeError(f"{key} has to be of type {dbt_command_config_annotations[key]}") - # if the param is not bool it must have a non null value - flag_prefix = '' - if param_value_type is bool and not value: - flag_prefix = 'no-' - cli_param_from_kwarg = "--" + flag_prefix + key.replace("_", "-") - command_params.append(cli_param_from_kwarg) - if param_value_type is str: - command_params.append(value) - elif param_value_type is int: - command_params.append(str(value)) - elif param_value_type is dict: - command_params.append(json.dumps(value)) - - return [dbt_bin, *command_params, command] + base_params = render_config(base_config) + command_params = render_config(command_config) + return [dbt_bin, *base_params, command, *command_params] class DbtBaseHook(BaseHook, ABC): diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 86bdbc5..7f2b865 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -280,10 +280,28 @@ def instantiate_hook(self): def execute(self, context: Any): """Runs the provided command in the provided execution environment""" self.instantiate_hook() + dbt_base_params = [ + 'log_format', 'version', 'use_colors', 'warn_error', + 'partial_parse', 'use_experimental_parser', 'profiles_dir' + ] + + dbt_base_config = { + key: val + for key, val in self.dbt_config.items() + if key in dbt_base_params + } + + dbt_command_config = { + key: val + for key, val in self.dbt_config.items() + if key not in dbt_base_params + } + self.dbt_cli_command = generate_dbt_cli_command( dbt_bin=self.dbt_bin, command=self.dbt_command, - **self.dbt_config + base_config=dbt_base_config, + command_config=dbt_command_config, ) self.dbt_hook.run_dbt(self.dbt_cli_command) From f2075f52a1a5d1831a6d3d97b867483360e16a96 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 14 Jan 2022 05:15:21 +0100 Subject: [PATCH 63/79] Allow a custom image other than the official from fishtown --- airflow_dbt/hooks/google.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index 29b2bbc..c067a8d 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -72,6 +72,7 @@ def __init__( gcs_staging_location: str = None, gcp_conn_id: str = "google_cloud_default", dbt_version: str = '0.21.0', + dbt_image: str = 'fishtownanalytics/dbt', env: Optional[Dict] = None, service_account: Optional[str] = None, ): @@ -87,6 +88,7 @@ def __init__( self.gcs_staging_blob = staging_blob self.dbt_version = dbt_version + self.dbt_image = dbt_image self.cloud_build_hook = CloudBuildHook(gcp_conn_id=gcp_conn_id) self.gcp_conn_id = gcp_conn_id self.project_id = project_id or self.cloud_build_hook.project_id @@ -109,7 +111,7 @@ def run_dbt(self, dbt_cmd: List[str]): cloud_build_config = { 'steps': [{ # use the official dbt docker image from dockerhub - 'name': f'fishtownanalytics/dbt:{self.dbt_version}', + 'name': f'{self.dbt_image}:{self.dbt_version}', 'args': dbt_cmd, 'env': [f'{k}={v}' for k, v in self.env.items()] }], From b27911256dde7bfbcfc79e146dd2709b5b0e264b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 14 Jan 2022 05:16:48 +0100 Subject: [PATCH 64/79] Add DbtCloudBuildOperator templated fields to the DbtBaseOperator ones --- airflow_dbt/operators/google.py | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/airflow_dbt/operators/google.py b/airflow_dbt/operators/google.py index 2e2e17b..878e934 100644 --- a/airflow_dbt/operators/google.py +++ b/airflow_dbt/operators/google.py @@ -1,12 +1,9 @@ -from typing import Dict, Optional - from airflow.models import BaseOperatorLink from airflow.plugins_manager import AirflowPlugin from airflow.utils.decorators import apply_defaults -from airflow_dbt.operators.dbt_operator import DbtBaseOperator -from airflow_dbt.dbt_command_config import DbtCommandConfig from airflow_dbt.hooks.google import DbtCloudBuildHook +from airflow_dbt.operators.dbt_operator import DbtBaseOperator class CloudBuildLogsLink(BaseOperatorLink): @@ -27,13 +24,9 @@ class CloudBuildLinkPlugin(AirflowPlugin): class DbtCloudBuildOperator(DbtBaseOperator): """Uses the CloudBuild Hook to run the operation in there by default""" - template_fields = [ - 'env', 'dbt_bin', 'command', 'config', 'gcs_staging_location', - 'project_id', 'dbt_version', 'service_account' - ] - operator_extra_links = [CloudBuildLogsLink] + # noinspection PyDeprecation @apply_defaults def __init__( self, @@ -56,6 +49,11 @@ def __init__( **kwargs ) + self.template_fields += [ + 'gcs_staging_location', 'project_id', 'dbt_version', + 'service_account' + ] + def instantiate_hook(self): """ Instantiates a Cloud build dbt hook. This has to be done out of the From 957f76b6a4049197b94dd6db91fd9babbcb23834 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 14 Jan 2022 05:53:00 +0100 Subject: [PATCH 65/79] Bump dbt docker image for cloud build --- airflow_dbt/hooks/google.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index c067a8d..6d9fb3f 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -5,7 +5,7 @@ from airflow.providers.google.cloud.hooks.cloud_build import CloudBuildHook from airflow.providers.google.cloud.hooks.gcs import GCSHook, _parse_gcs_url from airflow.providers.google.get_provider_info import get_provider_info -from airflow.settings import json +from airflow.utils.yaml import dump from packaging import version from airflow_dbt.hooks.base import DbtBaseHook @@ -71,7 +71,7 @@ def __init__( project_id: Optional[str] = None, gcs_staging_location: str = None, gcp_conn_id: str = "google_cloud_default", - dbt_version: str = '0.21.0', + dbt_version: str = '1.0.0', dbt_image: str = 'fishtownanalytics/dbt', env: Optional[Dict] = None, service_account: Optional[str] = None, @@ -112,7 +112,8 @@ def run_dbt(self, dbt_cmd: List[str]): 'steps': [{ # use the official dbt docker image from dockerhub 'name': f'{self.dbt_image}:{self.dbt_version}', - 'args': dbt_cmd, + 'entrypoint': 'bash', + 'args': ['-c'] + dbt_cmd, 'env': [f'{k}={v}' for k, v in self.env.items()] }], 'source': { @@ -135,17 +136,16 @@ def run_dbt(self, dbt_cmd: List[str]): f'{self.service_account}' cloud_build_config['serviceAccount'] = service_account_path_path - cloud_build_config_str = json.dumps(cloud_build_config, indent=2) logging.info( f'Running the following cloud build' - f' config:\n{cloud_build_config_str}' + f' config:\n{dump(cloud_build_config)}' ) build_results = self.cloud_build_hook.create_build( body=cloud_build_config, project_id=self.project_id, ) - + logging.info("Finished running") # print logs from GCS build_logs_blob = f'log-{build_results["id"]}.txt' with GCSHook().provide_file( @@ -158,7 +158,7 @@ def run_dbt(self, dbt_cmd: List[str]): logging.info(clean_line) # print result from build - logging.info('Build results:\n' + json.dumps(build_results, indent=2)) + logging.info('Build results:\n' + dump(build_results)) # set the log_url class param to be read from the "links" return build_results From a2a3346b12a385c36b0b2b9368326517376d423d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 14 Jan 2022 11:46:16 +0100 Subject: [PATCH 66/79] Cloud build hook can raise silently. Surround with a try/catch --- airflow_dbt/hooks/google.py | 52 ++++++++++++++++++++----------------- 1 file changed, 28 insertions(+), 24 deletions(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index 6d9fb3f..f48aaa7 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -30,7 +30,7 @@ def check_google_provider_version(version_min: str, version_max: str) -> None: version_max = version.parse(version_max) version_provider = version.parse(google_providers_version) if not version_min <= version_provider < version_max: - raise Exception( + raise ImportError( 'The provider "apache-airflow-providers-google" version "' f'{google_providers_version}" is not compatible with the current ' 'API. Please install a compatible version in the range ' @@ -112,9 +112,9 @@ def run_dbt(self, dbt_cmd: List[str]): 'steps': [{ # use the official dbt docker image from dockerhub 'name': f'{self.dbt_image}:{self.dbt_version}', - 'entrypoint': 'bash', - 'args': ['-c'] + dbt_cmd, - 'env': [f'{k}={v}' for k, v in self.env.items()] + 'entrypoint': dbt_cmd[0], + 'args': dbt_cmd[1:], + 'env': [f'{k}={v}' for k, v in self.env.items()], }], 'source': { 'storageSource': { @@ -141,26 +141,30 @@ def run_dbt(self, dbt_cmd: List[str]): f' config:\n{dump(cloud_build_config)}' ) - build_results = self.cloud_build_hook.create_build( - body=cloud_build_config, - project_id=self.project_id, - ) - logging.info("Finished running") - # print logs from GCS - build_logs_blob = f'log-{build_results["id"]}.txt' - with GCSHook().provide_file( - bucket_name=self.gcs_staging_bucket, - object_name=build_logs_blob - ) as log_file_handle: - for line in log_file_handle: - clean_line = line.decode('utf-8').strip() - if not clean_line == '': - logging.info(clean_line) - - # print result from build - logging.info('Build results:\n' + dump(build_results)) - # set the log_url class param to be read from the "links" - return build_results + try: + build_results = self.cloud_build_hook.create_build( + body=cloud_build_config, + project_id=self.project_id, + ) + + logging.info("Finished running: " + dump(build_results)) + # print logs from GCS + build_logs_blob = f'log-{build_results["id"]}.txt' + with GCSHook().provide_file( + bucket_name=self.gcs_staging_bucket, + object_name=build_logs_blob + ) as log_file_handle: + for line in log_file_handle: + clean_line = line.decode('utf-8').strip() + if clean_line: + logging.info(clean_line) + + # print result from build + logging.info('Build results:\n' + dump(build_results)) + # set the log_url class param to be read from the "links" + return build_results + except Exception as ex: + raise AirflowException("Exception running the build: ", str(ex)) def on_kill(self): """Stopping the build is not implemented until google providers v6""" From 4515199b17f9c4abf244510bac319174aeb8fb72 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Fri, 14 Jan 2022 11:47:33 +0100 Subject: [PATCH 67/79] Add default conn_id, dbt_version and dbt_image and simpler cloudbuild hook --- airflow_dbt/operators/google.py | 29 +++++++++++++---------------- 1 file changed, 13 insertions(+), 16 deletions(-) diff --git a/airflow_dbt/operators/google.py b/airflow_dbt/operators/google.py index 878e934..f3b4773 100644 --- a/airflow_dbt/operators/google.py +++ b/airflow_dbt/operators/google.py @@ -32,8 +32,9 @@ def __init__( self, gcs_staging_location: str, project_id: str = None, - gcp_conn_id: str = None, - dbt_version: str = None, + gcp_conn_id: str = "google_cloud_default", + dbt_version: str = '1.0.0', + dbt_image: str = 'fishtownanalytics/dbt', service_account: str = None, *args, **kwargs @@ -42,6 +43,7 @@ def __init__( self.gcp_conn_id = gcp_conn_id self.project_id = project_id self.dbt_version = dbt_version + self.dbt_image = dbt_image self.service_account = service_account super(DbtCloudBuildOperator, self).__init__( @@ -60,17 +62,12 @@ def instantiate_hook(self): constructor because by the time the constructor runs the params have not been yet interpolated. """ - hook_config = { - 'env': self.dbt_env, - 'gcs_staging_location': self.gcs_staging_location, - } - if self.project_id: - hook_config['project_id'] = self.project_id - if self.gcp_conn_id: - hook_config['gcp_conn_id'] = self.gcp_conn_id - if self.dbt_version: - hook_config['dbt_version'] = self.dbt_version - if self.service_account: - hook_config['service_account'] = self.service_account - - self.dbt_hook = DbtCloudBuildHook(**hook_config) + self.dbt_hook = DbtCloudBuildHook( + env=self.dbt_env, + gcs_staging_location=self.gcs_staging_location, + gcp_conn_id=self.gcp_conn_id, + dbt_version=self.dbt_version, + dbt_image=self.dbt_image, + service_account=self.service_account, + project_id=self.project_id, + ) From f42cdfef36796928e21e2493ea9e6330750849a1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 1 Feb 2022 06:11:34 +0100 Subject: [PATCH 68/79] Fix compoundd commands like 'dbt docs generate' not running in GCP --- airflow_dbt/hooks/base.py | 96 +++++---------------------------------- 1 file changed, 11 insertions(+), 85 deletions(-) diff --git a/airflow_dbt/hooks/base.py b/airflow_dbt/hooks/base.py index 747af77..d5f422d 100644 --- a/airflow_dbt/hooks/base.py +++ b/airflow_dbt/hooks/base.py @@ -8,7 +8,7 @@ from airflow_dbt.dbt_command_config import DbtCommandConfig -def render_config(config: dict[str, Union[str,bool]]) -> List[str]: +def render_config(config: dict[str, Union[str, bool]]) -> List[str]: """Renders a dictionary of options into a list of cli strings""" dbt_command_config_annotations = DbtCommandConfig.__annotations__ command_params = [] @@ -51,6 +51,11 @@ def generate_dbt_cli_command( Boolean flags must always be positive. Available params are: + :param command_config: Specific params for the commands + :type command_config: dict + :param base_config: Params that apply to the `dbt` program regardless of + the command it is running + :type base_config: dict :param command: The dbt sub-command to run :type command: str :param dbt_bin: Path to the dbt binary, defaults to `dbt` assumes it is @@ -60,88 +65,6 @@ def generate_dbt_cli_command( the base_command will be `run`. If any other flag not contemplated must be included it can also be added to this string :type command: str - :param version: Dbt version to use, in SEMVER. Defaults - to the last one '0.21.0' - :type version: str - :param record_timing_info: Dbt flag to add '--record-timing-info' - :type record_timing_info: bool - :param debug: Dbt flag to add '--debug' - :type debug: bool - :param log_format: Specifies how dbt's logs should be formatted. The - value for this flag can be one of: text, json, or default - :type log_format: str - :param write_json: If set to no it adds the `--no-write-json` Dbt flag - :type write_json: bool - :param strict: Only for use during dbt development. It performs extra - validation of dbt objects and internal consistency checks during - compilation - :type strict: bool - :param warn_error: Converts dbt warnings into errors - :type warn_error: bool - :param partial_parse: configure partial parsing in your project, and - will override the value set in `profiles.yml - :type partial_parse: bool - :param use_experimental_parser: Statically analyze model files in your - project and, if possible, extract needed information 3x faster than - a full Jinja render - :type use_experimental_parser: bool - :param use_colors: Displays colors in dbt logs - :type use_colors: bool - :param profiles_dir: Path to profiles.yaml dir. Can be relative from - the folder the DAG is being run, which usually is the home or de - DAGs folder - :type profiles_dir: str - :param project_dir: Path to the dbt project you want to run. Can be - relative to the path the DAG is being run - :type project_dir: str - :param profile: Which profile to load. Overrides setting in - dbt_project.yml - :type profile: Which profile to load. Overrides setting in - dbt_project.yml - :param target: Which target to load for the given profile - :type target: str - :param config_dir: Sames a profile_dir - :type config_dir: str - :param resource_type: One of: model,snapshot,source,analysis,seed, - exposure,test,default,all - :type resource_type: str - :param vars: Supply variables to the project. This argument overrides - variables defined in your dbt_project.yml file. This argument should - be a YAML string, eg. '{my_variable: my_value}' - :type vars: dict - :param full_refresh: If specified, dbt will drop incremental models and - fully-recalculate the incremental table from the model definition - :type full_refresh: bool - :param data: Run data tests defined in "tests" directory. - :type data: bool - :param schema: Run constraint validations from schema.yml files - :type schema: bool - :param models: Flag used to choose a node or subset of nodes to apply - the command to (v0.210.0 and lower) - :type models: str - :param exclude: Nodes to exclude from the set defined with - select/models - :type exclude: str - :param select: Flag used to choose a node or subset of nodes to apply - the command to (v0.21.0 and higher) - :type select: str - :param selector: Config param to reference complex selects defined in - the config yaml - :type selector: str - :param output: {json,name,path,selector} - :type output: str - :param output_keys: Which keys to output - :type output_keys: str - :param host: Specify the host to listen on for the rpc server - :type host: str - :param port: Specify the port number for the rpc server - :type port: int - :param fail_fast: Stop execution upon a first test failure - :type fail_fast: bool - :param args: - :type args: - :param no_compile: Do not run "dbt compile" as part of docs generation - :type no_compile: bool """ if not dbt_bin: raise ValueError("dbt_bin is mandatory") @@ -149,12 +72,15 @@ def generate_dbt_cli_command( raise ValueError("command mandatory") base_params = render_config(base_config) command_params = render_config(command_config) - return [dbt_bin, *base_params, command, *command_params] + # commands like 'dbt docs generate' need the command to be split in two + command_pieces = command.split(" ") + return [dbt_bin, *base_params, *command_pieces, *command_params] class DbtBaseHook(BaseHook, ABC): """ - Simple wrapper around the dbt CLI and interface to implement dbt hooks + Base abstract class for all DbtHooks to have a common interface and force + implement the mandatory `run_dbt()` function. """ def __init__(self, env: Optional[Dict] = None): From 0c1bcf4cf8129a8d323e242e96ebd1a9c3bae986 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 1 Feb 2022 06:13:39 +0100 Subject: [PATCH 69/79] Export artifacts (`target` folder) when running in cloud build: This includes `dbt docs generate` --- airflow_dbt/hooks/google.py | 117 +++++++++++++++++++++----------- airflow_dbt/operators/google.py | 33 +++------ 2 files changed, 86 insertions(+), 64 deletions(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index f48aaa7..2a2b9bb 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -1,8 +1,12 @@ +"""Provides hooks and helper functions to allow running dbt in GCP""" + import logging +from pathlib import Path from typing import Any, Dict, List, Optional from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.cloud_build import CloudBuildHook +# noinspection PyProtectedMember from airflow.providers.google.cloud.hooks.gcs import GCSHook, _parse_gcs_url from airflow.providers.google.get_provider_info import get_provider_info from airflow.utils.yaml import dump @@ -38,6 +42,8 @@ def check_google_provider_version(version_min: str, version_max: str) -> None: ) +# if the Google provider available is not within the versions the library will +# raise an exception check_google_provider_version( version_min=MIN_AIRFLOW_GOOGLE_PROVIDER_VERSION, version_max=MAX_AIRFLOW_GOOGLE_PROVIDER_VERSION, @@ -46,53 +52,69 @@ def check_google_provider_version(version_min: str, version_max: str) -> None: class DbtCloudBuildHook(DbtBaseHook): """ - Runs the dbt command in a Cloud Build job in GCP - - :type env: dict - :param env: If set, passed to the dbt executor - :param project_id: GCP Project ID as stated in the console - :type project_id: str - :param gcp_conn_id: The connection ID to use when fetching connection info. - :type gcp_conn_id: str - :param gcs_staging_location: Where to store the sources to be fetch later - by the cloud build job. It should be the GCS url for a folder. For - example: `gs://my-bucket/stored. A sub-folder will be generated to - avoid collision between possible different concurrent runs. - :type gcs_staging_location: str - :param dbt_version: the DBT version to be fetched from dockerhub. Defaults - to '0.21.0' - :type dbt_version: str - :param service_account: email for the service account. If set must be - accompanied by the project_id + Connects to GCP Cloud Build, creates a build config, submits it and waits + for results. """ - def __init__( + def __init__( self, project_id: Optional[str] = None, gcs_staging_location: str = None, gcp_conn_id: str = "google_cloud_default", - dbt_version: str = '1.0.0', - dbt_image: str = 'fishtownanalytics/dbt', env: Optional[Dict] = None, service_account: Optional[str] = None, + dbt_version: str = '1.0.0', + dbt_image: str = 'fishtownanalytics/dbt', + dbt_project_dir: str = None, + dbt_artifacts_dest: str = None, ): + """ + Runs the dbt command in a Cloud Build job in GCP + + :param dbt_artifacts_dest: Folder in GCS destination for the artifacts. + For example `gs://my-bucket/path/to/artifacts/` + :type dbt_artifacts_dest: str + :type env: dict + :param env: If set, passed to the dbt executor + :param project_id: GCP Project ID as stated in the console + :type project_id: str + :param gcp_conn_id: The connection ID to use when fetching connection + info. + :type gcp_conn_id: str + :param gcs_staging_location: Where to store the sources to be fetched + later by the cloud build job. It should be the GCS url of a folder. + For example: `gs://my-bucket/stored. A sub-folder will be generated + to avoid collision between possible different concurrent runs. + :type gcs_staging_location: str + :param dbt_version: the DBT version to be fetched from dockerhub. + Defaults to '1.0.0'. It represents the image tag. So it must also be + a tag for your custom Docker dbt image if you provide one. + :type dbt_version: str + :param service_account: email for the service account. If set must be + accompanied by the project_id + + + """ staging_bucket, staging_blob = _parse_gcs_url(gcs_staging_location) # we have provided something similar to # 'gs:///' - if not staging_blob.endswith('.tar.gz'): + if Path(staging_blob).suffix not in ['.gz', '.gzip']: raise AirflowException( - f'The provided blob "{staging_blob}" to a compressed file does not ' + - 'have the right extension ".tar.gz"' + f'The provided blob "{staging_blob}" to a compressed file ' + 'does not have the right extension ".tar.gz" or ".gzip"' ) + # gcp config self.gcs_staging_bucket = staging_bucket self.gcs_staging_blob = staging_blob - - self.dbt_version = dbt_version - self.dbt_image = dbt_image self.cloud_build_hook = CloudBuildHook(gcp_conn_id=gcp_conn_id) self.gcp_conn_id = gcp_conn_id self.project_id = project_id or self.cloud_build_hook.project_id self.service_account = service_account + # dbt config + self.dbt_version = dbt_version + self.dbt_image = dbt_image + self.dbt_project_dir = dbt_project_dir + self.dbt_artifacts_dest = dbt_artifacts_dest super().__init__(env=env) @@ -100,22 +122,14 @@ def get_conn(self) -> Any: """Returns the cloud build connection, which is a gcp connection""" return self.cloud_build_hook.get_conn() - def run_dbt(self, dbt_cmd: List[str]): - """ - Run the dbt command. In version 5 of the providers - - :param dbt_cmd: The dbt whole command to run - :type dbt_cmd: List[str] - """ - # See: https://cloud.google.com/cloud-build/docs/api/reference/rest/v1/projects.builds + def _get_cloud_build_config(self, dbt_cmd: List[str]) -> Dict: cloud_build_config = { 'steps': [{ - # use the official dbt docker image from dockerhub 'name': f'{self.dbt_image}:{self.dbt_version}', 'entrypoint': dbt_cmd[0], 'args': dbt_cmd[1:], 'env': [f'{k}={v}' for k, v in self.env.items()], - }], + }, ], 'source': { 'storageSource': { "bucket": self.gcs_staging_bucket, @@ -132,9 +146,32 @@ def run_dbt(self, dbt_cmd: List[str]): } if self.service_account: - service_account_path_path = f'projects/{self.project_id}/serviceAccounts/' \ - f'{self.service_account}' - cloud_build_config['serviceAccount'] = service_account_path_path + cloud_build_config['serviceAccount'] = ( + f'projects/{self.project_id}/serviceAccounts/' + f'{self.service_account}' + ) + + if self.dbt_artifacts_dest: + cloud_build_config['steps'].append({ + 'name': 'gcr.io/cloud-builders/gsutil', + 'args': ['-m', 'cp', '-r', + f'{self.dbt_project_dir}/target/**', + self.dbt_artifacts_dest] + }) + + return cloud_build_config + + def run_dbt(self, dbt_cmd: List[str]): + """ + Run the dbt command. In version 5 of the providers + + :param dbt_cmd: The dbt whole command to run + :type dbt_cmd: List[str] + """ + # See: https://cloud.google.com/cloud-build/docs/api/reference/rest + # /v1/projects.builds + + cloud_build_config = self._get_cloud_build_config(dbt_cmd) logging.info( f'Running the following cloud build' diff --git a/airflow_dbt/operators/google.py b/airflow_dbt/operators/google.py index f3b4773..c8ddedf 100644 --- a/airflow_dbt/operators/google.py +++ b/airflow_dbt/operators/google.py @@ -1,30 +1,16 @@ -from airflow.models import BaseOperatorLink -from airflow.plugins_manager import AirflowPlugin from airflow.utils.decorators import apply_defaults from airflow_dbt.hooks.google import DbtCloudBuildHook from airflow_dbt.operators.dbt_operator import DbtBaseOperator -class CloudBuildLogsLink(BaseOperatorLink): - """Add a link to the logs generated from a build in cloud build""" - name = "Cloud Build Logs" - - def get_link(self, operator, _): - """Returns the log url for the cloud build logs stored as class prop""" - return operator.log_url - - -class CloudBuildLinkPlugin(AirflowPlugin): - """Registers the extra links""" - name = "cloud_build_link_plugin" - operator_extra_links = [CloudBuildLogsLink()] - - class DbtCloudBuildOperator(DbtBaseOperator): - """Uses the CloudBuild Hook to run the operation in there by default""" + """Uses the CloudBuild Hook to run the provided dbt config""" - operator_extra_links = [CloudBuildLogsLink] + template_fields = DbtBaseOperator.template_fields + [ + 'gcs_staging_location', 'project_id', 'dbt_version', + 'service_account', 'dbt_artifacts_dest' + ] # noinspection PyDeprecation @apply_defaults @@ -35,10 +21,12 @@ def __init__( gcp_conn_id: str = "google_cloud_default", dbt_version: str = '1.0.0', dbt_image: str = 'fishtownanalytics/dbt', + dbt_artifacts_dest: str = None, service_account: str = None, *args, **kwargs ): + self.dbt_artifacts_dest = dbt_artifacts_dest self.gcs_staging_location = gcs_staging_location self.gcp_conn_id = gcp_conn_id self.project_id = project_id @@ -51,11 +39,6 @@ def __init__( **kwargs ) - self.template_fields += [ - 'gcs_staging_location', 'project_id', 'dbt_version', - 'service_account' - ] - def instantiate_hook(self): """ Instantiates a Cloud build dbt hook. This has to be done out of the @@ -70,4 +53,6 @@ def instantiate_hook(self): dbt_image=self.dbt_image, service_account=self.service_account, project_id=self.project_id, + dbt_project_dir=self.dbt_config.get('project_dir'), + dbt_artifacts_dest=self.dbt_artifacts_dest, ) From 697a2ba3bc2094cfd1f2de1ea8eb1dad7d15795f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 1 Feb 2022 06:14:16 +0100 Subject: [PATCH 70/79] Enhance type safety --- airflow_dbt/operators/dbt_operator.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 7f2b865..5141e26 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -31,7 +31,7 @@ class DbtBaseOperator(BaseOperator): dbt_env: Dict dbt_bin: str dbt_command: str - dbt_config: Dict + dbt_config: DbtCommandConfig dbt_hook: DbtBaseHook dbt_cli_command: List[str] From 460d3617dcb1824136f8e1eaa1427cd7d10d421c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 8 Mar 2022 06:09:17 +0100 Subject: [PATCH 71/79] Use `Dict` instead of the default new `dict` for backwards compat --- airflow_dbt/hooks/base.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow_dbt/hooks/base.py b/airflow_dbt/hooks/base.py index d5f422d..66936a8 100644 --- a/airflow_dbt/hooks/base.py +++ b/airflow_dbt/hooks/base.py @@ -8,7 +8,7 @@ from airflow_dbt.dbt_command_config import DbtCommandConfig -def render_config(config: dict[str, Union[str, bool]]) -> List[str]: +def render_config(config: Dict[str, Union[str, bool]]) -> List[str]: """Renders a dictionary of options into a list of cli strings""" dbt_command_config_annotations = DbtCommandConfig.__annotations__ command_params = [] From 7cab0fd772429eeb30208586f88a9ee8d9593e90 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 8 Mar 2022 06:11:25 +0100 Subject: [PATCH 72/79] use the cloud build library instead of the hook --- airflow_dbt/hooks/google.py | 128 +++++++++++++++++------------------- 1 file changed, 59 insertions(+), 69 deletions(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index 2a2b9bb..1dbdefb 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -1,54 +1,23 @@ """Provides hooks and helper functions to allow running dbt in GCP""" import logging +import traceback from pathlib import Path from typing import Any, Dict, List, Optional from airflow.exceptions import AirflowException -from airflow.providers.google.cloud.hooks.cloud_build import CloudBuildHook -# noinspection PyProtectedMember +from airflow.providers.google.cloud.hooks.cloud_build import ( + CloudBuildHook, +) from airflow.providers.google.cloud.hooks.gcs import GCSHook, _parse_gcs_url -from airflow.providers.google.get_provider_info import get_provider_info from airflow.utils.yaml import dump -from packaging import version +from google.api_core.operation import Operation +from google.cloud.devtools.cloudbuild_v1 import ( + Build, +) from airflow_dbt.hooks.base import DbtBaseHook -MIN_AIRFLOW_GOOGLE_PROVIDER_VERSION = '5.0.0' -MAX_AIRFLOW_GOOGLE_PROVIDER_VERSION = '6.0.0' - - -def check_google_provider_version(version_min: str, version_max: str) -> None: - """ - Check we're using the right Google provider version. As Cloud Composer is - the most broadly used Airflow installation we will default to the latest - version composer is using - - :param version_min: Minimum version of the Google provider in semver format - :type version_min: str - :param version_max: Maximum version of the Google provider in semver format - :type version_max: str - """ - google_providers_version = get_provider_info().get('versions')[0] - version_min = version.parse(version_min) - version_max = version.parse(version_max) - version_provider = version.parse(google_providers_version) - if not version_min <= version_provider < version_max: - raise ImportError( - 'The provider "apache-airflow-providers-google" version "' - f'{google_providers_version}" is not compatible with the current ' - 'API. Please install a compatible version in the range ' - f'>={version_min}, <{version_max}"' - ) - - -# if the Google provider available is not within the versions the library will -# raise an exception -check_google_provider_version( - version_min=MIN_AIRFLOW_GOOGLE_PROVIDER_VERSION, - version_max=MAX_AIRFLOW_GOOGLE_PROVIDER_VERSION, -) - class DbtCloudBuildHook(DbtBaseHook): """ @@ -98,7 +67,7 @@ def __init__( staging_bucket, staging_blob = _parse_gcs_url(gcs_staging_location) # we have provided something similar to # 'gs:///' - if Path(staging_blob).suffix not in ['.gz', '.gzip']: + if Path(staging_blob).suffix not in ['.gz', '.gzip', '.zip']: raise AirflowException( f'The provided blob "{staging_blob}" to a compressed file ' 'does not have the right extension ".tar.gz" or ".gzip"' @@ -129,35 +98,38 @@ def _get_cloud_build_config(self, dbt_cmd: List[str]) -> Dict: 'entrypoint': dbt_cmd[0], 'args': dbt_cmd[1:], 'env': [f'{k}={v}' for k, v in self.env.items()], - }, ], + }], 'source': { - 'storageSource': { + 'storage_source': { "bucket": self.gcs_staging_bucket, - "object": self.gcs_staging_blob, + "object_": self.gcs_staging_blob, } }, 'options': { # default is legacy and its behaviour is subject to change 'logging': 'GCS_ONLY', }, - # mandatory if using a service_account, it also is relevant as - # transactional data - 'logsBucket': self.gcs_staging_bucket, + 'logs_bucket': self.gcs_staging_bucket, } if self.service_account: - cloud_build_config['serviceAccount'] = ( + cloud_build_config['service_account'] = ( f'projects/{self.project_id}/serviceAccounts/' f'{self.service_account}' ) if self.dbt_artifacts_dest: - cloud_build_config['steps'].append({ + # ensure the path ends with a slash as it should if it's a folder + gcs_dest_url = self.dbt_artifacts_dest.lstrip('/') + '/' + artifacts_step = { 'name': 'gcr.io/cloud-builders/gsutil', - 'args': ['-m', 'cp', '-r', + 'args': [ + '-m', 'cp', '-r', f'{self.dbt_project_dir}/target/**', - self.dbt_artifacts_dest] - }) + gcs_dest_url + ] + } + cloud_build_config['steps'].append(artifacts_step) return cloud_build_config @@ -170,37 +142,55 @@ def run_dbt(self, dbt_cmd: List[str]): """ # See: https://cloud.google.com/cloud-build/docs/api/reference/rest # /v1/projects.builds - cloud_build_config = self._get_cloud_build_config(dbt_cmd) - logging.info( f'Running the following cloud build' f' config:\n{dump(cloud_build_config)}' ) try: - build_results = self.cloud_build_hook.create_build( - body=cloud_build_config, - project_id=self.project_id, + cloud_build_client = self.get_conn() + + self.log.info("Start creating build.") + + operation: Operation = cloud_build_client.create_build( + request={ + 'project_id': self.project_id, + 'build': cloud_build_config + } + ) + # wait for the operation to complete + operation.result() + + result_build: Build = operation.metadata.build + + self.log.info( + f"Build has been created: {result_build.id}.\n" + f'Build logs available at: {result_build.log_url} and the ' + f'file gs://{result_build.logs_bucket}/log-' + f'{result_build.id}.txt' ) - logging.info("Finished running: " + dump(build_results)) # print logs from GCS - build_logs_blob = f'log-{build_results["id"]}.txt' with GCSHook().provide_file( - bucket_name=self.gcs_staging_bucket, - object_name=build_logs_blob + bucket_name=result_build.logs_bucket, + object_name=f'log-{result_build.id}.txt', ) as log_file_handle: - for line in log_file_handle: - clean_line = line.decode('utf-8').strip() - if clean_line: - logging.info(clean_line) - - # print result from build - logging.info('Build results:\n' + dump(build_results)) - # set the log_url class param to be read from the "links" - return build_results + clean_lines = [ + line.decode('utf-8').strip() + for line in log_file_handle if line + ] + log_block = '\n'.join(clean_lines) + hr = '-' * 80 + logging.info( + f'Logs from the build {result_build.id}:\n' + f'{hr}\n' + f'{log_block}\n' + f'{hr}' + ) + return result_build except Exception as ex: + traceback.print_exc() raise AirflowException("Exception running the build: ", str(ex)) def on_kill(self): From b3dbdb6a0a682e1376dff13346e279922f5e98f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 8 Mar 2022 06:12:20 +0100 Subject: [PATCH 73/79] Ignore terraform, dbt and pycharm related files --- .gitignore | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.gitignore b/.gitignore index 762cc89..9ca4abe 100644 --- a/.gitignore +++ b/.gitignore @@ -133,3 +133,9 @@ dmypy.json # pytype static type analyzer .pytype/ + +.run/ + +.terraform/ + +.user.yaml \ No newline at end of file From e1969f83ad5c18dd4b050bb762728eaa42b61b0b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Tue, 8 Mar 2022 06:12:36 +0100 Subject: [PATCH 74/79] Bump version --- airflow_dbt/__version__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow_dbt/__version__.py b/airflow_dbt/__version__.py index e2b2363..e6e03c6 100644 --- a/airflow_dbt/__version__.py +++ b/airflow_dbt/__version__.py @@ -1,3 +1,3 @@ -VERSION = (0, 5, 0) +VERSION = (0, 5, 7) __version__ = '.'.join(map(str, VERSION)) From 9766bbcdc8fbddd15993e61d3204153cd7bb2fd0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 24 Oct 2022 19:33:59 +0200 Subject: [PATCH 75/79] chore: update default dbt-imge to github registry --- airflow_dbt/hooks/google.py | 41 +++++++++++++++---------------------- 1 file changed, 16 insertions(+), 25 deletions(-) diff --git a/airflow_dbt/hooks/google.py b/airflow_dbt/hooks/google.py index 1dbdefb..305127e 100644 --- a/airflow_dbt/hooks/google.py +++ b/airflow_dbt/hooks/google.py @@ -11,7 +11,6 @@ ) from airflow.providers.google.cloud.hooks.gcs import GCSHook, _parse_gcs_url from airflow.utils.yaml import dump -from google.api_core.operation import Operation from google.cloud.devtools.cloudbuild_v1 import ( Build, ) @@ -29,11 +28,11 @@ def __init__( self, project_id: Optional[str] = None, gcs_staging_location: str = None, - gcp_conn_id: str = "google_cloud_default", + gcp_conn_id: str = None, env: Optional[Dict] = None, service_account: Optional[str] = None, - dbt_version: str = '1.0.0', - dbt_image: str = 'fishtownanalytics/dbt', + dbt_version: str = 'latest', + dbt_image: str = 'ghcr.io/dbt-labs/dbt-bigquery', dbt_project_dir: str = None, dbt_artifacts_dest: str = None, ): @@ -56,7 +55,7 @@ def __init__( to avoid collision between possible different concurrent runs. :type gcs_staging_location: str :param dbt_version: the DBT version to be fetched from dockerhub. - Defaults to '1.0.0'. It represents the image tag. So it must also be + Defaults to 'latest'. It represents the image tag. So it must also be a tag for your custom Docker dbt image if you provide one. :type dbt_version: str :param service_account: email for the service account. If set must be @@ -75,7 +74,10 @@ def __init__( # gcp config self.gcs_staging_bucket = staging_bucket self.gcs_staging_blob = staging_blob - self.cloud_build_hook = CloudBuildHook(gcp_conn_id=gcp_conn_id) + if gcp_conn_id is None: + self.cloud_build_hook = CloudBuildHook() + else: + self.cloud_build_hook = CloudBuildHook(gcp_conn_id=gcp_conn_id) self.gcp_conn_id = gcp_conn_id self.project_id = project_id or self.cloud_build_hook.project_id self.service_account = service_account @@ -140,8 +142,7 @@ def run_dbt(self, dbt_cmd: List[str]): :param dbt_cmd: The dbt whole command to run :type dbt_cmd: List[str] """ - # See: https://cloud.google.com/cloud-build/docs/api/reference/rest - # /v1/projects.builds + # See: https://cloud.google.com/cloud-build/docs/api/reference/rest/v1/projects.builds cloud_build_config = self._get_cloud_build_config(dbt_cmd) logging.info( f'Running the following cloud build' @@ -149,28 +150,18 @@ def run_dbt(self, dbt_cmd: List[str]): ) try: - cloud_build_client = self.get_conn() - - self.log.info("Start creating build.") - - operation: Operation = cloud_build_client.create_build( - request={ - 'project_id': self.project_id, - 'build': cloud_build_config - } - ) - # wait for the operation to complete - operation.result() - - result_build: Build = operation.metadata.build - + # cloud_build_client = self.get_conn() + self.log.info("Creating build") + result_build: Build = self.cloud_build_hook.create_build( + cloud_build_config + ) self.log.info( f"Build has been created: {result_build.id}.\n" f'Build logs available at: {result_build.log_url} and the ' f'file gs://{result_build.logs_bucket}/log-' f'{result_build.id}.txt' ) - + self.build_id = result_build.id # print logs from GCS with GCSHook().provide_file( bucket_name=result_build.logs_bucket, @@ -195,4 +186,4 @@ def run_dbt(self, dbt_cmd: List[str]): def on_kill(self): """Stopping the build is not implemented until google providers v6""" - raise NotImplementedError + self.cloud_build_hook.cancel_build(self.build_id) From f1744a877b4d3c7b2a161d56018b2da5d9a973d3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 24 Oct 2022 19:34:11 +0200 Subject: [PATCH 76/79] feat: add debug operator --- airflow_dbt/operators/dbt_operator.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/airflow_dbt/operators/dbt_operator.py b/airflow_dbt/operators/dbt_operator.py index 5141e26..84f85e1 100644 --- a/airflow_dbt/operators/dbt_operator.py +++ b/airflow_dbt/operators/dbt_operator.py @@ -367,3 +367,12 @@ class DbtCleanOperator(DbtBaseOperator): @apply_defaults def __init__(self, *args, **kwargs): super().__init__(*args, command='clean', **kwargs) + + +class DbtDebugOperator(DbtBaseOperator): + """Runs a dbt clean command""" + + # noinspection PyDeprecation + @apply_defaults + def __init__(self, *args, **kwargs): + super().__init__(*args, command='debug', **kwargs) From f34d22eba1bf00affe6781c2dd9394e0ce90b999 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 24 Oct 2022 19:35:07 +0200 Subject: [PATCH 77/79] chore: update default dbt docker image versions, part 2 --- airflow_dbt/operators/google.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow_dbt/operators/google.py b/airflow_dbt/operators/google.py index c8ddedf..dd83921 100644 --- a/airflow_dbt/operators/google.py +++ b/airflow_dbt/operators/google.py @@ -19,8 +19,8 @@ def __init__( gcs_staging_location: str, project_id: str = None, gcp_conn_id: str = "google_cloud_default", - dbt_version: str = '1.0.0', - dbt_image: str = 'fishtownanalytics/dbt', + dbt_version: str = '1.3.latest', + dbt_image: str = 'ghcr.io/dbt-labs/dbt-bigquery', dbt_artifacts_dest: str = None, service_account: str = None, *args, From 09548af8bfd8e70356be629a756a828a0bdb3e30 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 24 Oct 2022 19:35:30 +0200 Subject: [PATCH 78/79] chore: simpler and standard __version__ so that compilation picks it --- airflow_dbt/__version__.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/airflow_dbt/__version__.py b/airflow_dbt/__version__.py index e6e03c6..db55ef1 100644 --- a/airflow_dbt/__version__.py +++ b/airflow_dbt/__version__.py @@ -1,3 +1 @@ -VERSION = (0, 5, 7) - -__version__ = '.'.join(map(str, VERSION)) +__version__ = "0.5.10" From 98692ed0f859ab0c4a818ef2d531927eb267f170 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20I=C3=B1igo?= Date: Mon, 24 Oct 2022 19:36:57 +0200 Subject: [PATCH 79/79] ci: replace old build system with the new default pyproject --- .gitignore | 4 ++- pyproject.toml | 32 +++++++++++++++++++ setup.py | 84 -------------------------------------------------- 3 files changed, 35 insertions(+), 85 deletions(-) create mode 100644 pyproject.toml delete mode 100644 setup.py diff --git a/.gitignore b/.gitignore index 9ca4abe..f2b3eaf 100644 --- a/.gitignore +++ b/.gitignore @@ -138,4 +138,6 @@ dmypy.json .terraform/ -.user.yaml \ No newline at end of file +.user.yaml + +.idea/ diff --git a/pyproject.toml b/pyproject.toml new file mode 100644 index 0000000..4ed21dc --- /dev/null +++ b/pyproject.toml @@ -0,0 +1,32 @@ +[build-system] +requires = ["hatchling"] +build-backend = "hatchling.build" + +[project] +name = "airflow-dbt-dinigo" +dynamic = ["version"] +description = "Apache Airflow integration for dbt" +readme = "README.md" +license = "MIT" +classifiers = [ + "Development Status :: 5 - Production/Stable", + "License :: OSI Approved :: MIT License", + "Operating System :: MacOS :: MacOS X", + "Operating System :: POSIX :: Linux", + "Programming Language :: Python :: 3.7", +] +dependencies = [ + "apache-airflow >= 1.10.3", +] +packages = [ + { include = "airflow_dbt" }, +] + +[project.optional-dependencies] +google = [ + "apache-airflow-providers-google", + "google-cloud-build", +] + +[tool.hatch.version] +path = "airflow_dbt/__version__.py" diff --git a/setup.py b/setup.py deleted file mode 100644 index 325e51b..0000000 --- a/setup.py +++ /dev/null @@ -1,84 +0,0 @@ -import io -import os -import sys -from shutil import rmtree -from setuptools import setup, find_packages, Command - -here = os.path.abspath(os.path.dirname(__file__)) - -# Load the package's __version__.py module as a dictionary. -about = {} -with open(os.path.join(here, 'airflow_dbt', '__version__.py')) as f: - exec(f.read(), about) - -with io.open(os.path.join(here, 'README.md'), encoding='utf-8') as f: - long_description = f.read() - - -class UploadCommand(Command): - """Support setup.py upload.""" - - description = 'Build and publish the package.' - user_options = [] - - @staticmethod - def status(s): - """Prints things in bold.""" - print('\033[1m{0}\033[0m'.format(s)) - - def initialize_options(self): - pass - - def finalize_options(self): - pass - - def run(self): - try: - self.status('Removing previous builds…') - rmtree(os.path.join(here, 'dist')) - except OSError: - pass - - self.status('Building Source and Wheel (universal) distribution…') - os.system('{0} setup.py sdist bdist_wheel --universal'.format(sys.executable)) - - self.status('Uploading the package to PyPI via Twine…') - os.system('twine upload dist/*') - - self.status('Pushing git tags…') - os.system('git tag v{0}'.format(about['__version__'])) - os.system('git push --tags') - - sys.exit() - - -setup( - name='airflow_dbt', - version=about['__version__'], - packages=find_packages(exclude=['tests']), - install_requires=['apache-airflow >= 1.10.3'], - author='GoCardless', - author_email='engineering@gocardless.com', - description='Apache Airflow integration for dbt', - long_description=long_description, - long_description_content_type='text/markdown', - license='MIT', - url='https://github.com/gocardless/airflow-dbt', - classifiers=[ - 'Development Status :: 5 - Production/Stable', - - 'License :: OSI Approved :: MIT License', - - 'Operating System :: MacOS :: MacOS X', - 'Operating System :: POSIX :: Linux', - - 'Programming Language :: Python :: 3.7', - ], - # $ setup.py upload support. - cmdclass={ - 'upload': UploadCommand, - }, - extras_require={ - 'google': 'apache-airflow-providers-google==5.0.0' - }, -)