Skip to content

Commit 0841817

Browse files
authored
Improve airflow integration (#11)
1 parent 6e777b0 commit 0841817

File tree

3 files changed

+44
-12
lines changed

3 files changed

+44
-12
lines changed

opendbt/airflow/__init__.py

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -60,8 +60,8 @@ def load_dbt_tasks(self,
6060
end_node: BaseOperator = None,
6161
tag: str = None,
6262
resource_type="all",
63-
run_dbt_seeds=False,
64-
run_singular_tests=False) -> Tuple[BaseOperator, BaseOperator]:
63+
include_dbt_seeds=False,
64+
include_singular_tests=False) -> Tuple[BaseOperator, BaseOperator]:
6565
"""
6666
This method is used to add dbt tasks to Given DAG.
6767
@@ -81,7 +81,7 @@ def load_dbt_tasks(self,
8181
dag=dag)
8282
end_node = end_node if end_node else EmptyOperator(task_id='dbt-%s-end' % self.project_dir.name, dag=dag)
8383

84-
if run_dbt_seeds:
84+
if include_dbt_seeds:
8585
# add dbt seeds job after start node abd before all other dbt jobs
8686
first_node = start_node
8787
start_node = OpenDbtExecutorOperator(dag=dag,
@@ -106,7 +106,7 @@ def load_dbt_tasks(self,
106106
if resource_type == "test" and not str(node.name).startswith("source_"):
107107
if node.resource_type == "test":
108108
dbt_tasks[node.unique_id] = OpenDbtExecutorOperator(dag=dag,
109-
task_id=node.unique_id.rsplit('.', 1)[0],
109+
task_id=node.unique_id,
110110
project_dir=self.project_dir,
111111
profiles_dir=self.profiles_dir,
112112
target=self.target,
@@ -132,7 +132,7 @@ def load_dbt_tasks(self,
132132
# we are skipping model tests because they are included above with model execution( `build` command)
133133
# source table tests
134134
dbt_tasks[node.unique_id] = OpenDbtExecutorOperator(dag=dag,
135-
task_id=node.unique_id.rsplit('.', 1)[0],
135+
task_id=node.unique_id,
136136
project_dir=self.project_dir,
137137
profiles_dir=self.profiles_dir,
138138
target=self.target,
@@ -153,7 +153,7 @@ def load_dbt_tasks(self,
153153
task.set_upstream(dbt_tasks[upstream_id])
154154

155155
singular_tests = None
156-
if run_singular_tests:
156+
if include_singular_tests:
157157
singular_tests = OpenDbtExecutorOperator(dag=dag,
158158
task_id=f"{self.project_dir.name}_singular_tests",
159159
project_dir=self.project_dir,
@@ -167,7 +167,7 @@ def load_dbt_tasks(self,
167167
# set downstream dependencies for the end nodes.
168168
self.log.debug(f"Setting downstream of {task.task_id} -> {end_node.task_id}")
169169

170-
if run_singular_tests and singular_tests:
170+
if include_singular_tests and singular_tests:
171171
task.set_downstream(singular_tests)
172172
else:
173173
task.set_downstream(end_node)
@@ -177,6 +177,6 @@ def load_dbt_tasks(self,
177177
self.log.debug(f"Setting upstream of {task.task_id} -> {start_node}")
178178
task.set_upstream(start_node)
179179

180-
if run_singular_tests:
180+
if include_singular_tests:
181181
singular_tests.set_downstream(end_node)
182182
return start_node, end_node
Lines changed: 32 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,32 @@
1+
from pathlib import Path
2+
3+
from airflow import DAG
4+
from airflow.operators.empty import EmptyOperator
5+
from airflow.utils.dates import days_ago
6+
7+
from opendbt.airflow import OpenDbtAirflowProject
8+
9+
default_args = {
10+
'owner': 'airflow',
11+
'depends_on_past': False,
12+
'email_on_failure': False,
13+
'email_on_retry': False,
14+
'retries': 1
15+
}
16+
17+
with DAG(
18+
dag_id='dbt_tests_workflow',
19+
default_args=default_args,
20+
description='DAG To run dbt tests',
21+
schedule_interval=None,
22+
start_date=days_ago(3),
23+
catchup=False,
24+
max_active_runs=1
25+
) as dag:
26+
start = EmptyOperator(task_id="start")
27+
end = EmptyOperator(task_id="end")
28+
29+
DBT_PROJ_DIR = Path("/opt/dbttest")
30+
31+
p = OpenDbtAirflowProject(project_dir=DBT_PROJ_DIR, profiles_dir=DBT_PROJ_DIR, target='dev')
32+
p.load_dbt_tasks(dag=dag, start_node=start, end_node=end, resource_type='test')

tests/resources/airflow/dags/dbt_workflow.py

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1,9 +1,9 @@
11
from pathlib import Path
22

3+
from airflow import DAG
34
from airflow.operators.empty import EmptyOperator
45
from airflow.utils.dates import days_ago
56

6-
from airflow import DAG
77
from opendbt.airflow import OpenDbtAirflowProject
88

99
default_args = {
@@ -26,7 +26,7 @@
2626
start = EmptyOperator(task_id="start")
2727
end = EmptyOperator(task_id="end")
2828

29-
DBTTEST_DIR = Path("/opt/dbttest")
29+
DBT_PROJ_DIR = Path("/opt/dbttest")
3030

31-
p = OpenDbtAirflowProject(project_dir=DBTTEST_DIR, profiles_dir=DBTTEST_DIR, target='dev')
32-
p.load_dbt_tasks(dag=dag, start_node=start, end_node=end)
31+
p = OpenDbtAirflowProject(project_dir=DBT_PROJ_DIR, profiles_dir=DBT_PROJ_DIR, target='dev')
32+
p.load_dbt_tasks(dag=dag, start_node=start, end_node=end, include_singular_tests=True, include_dbt_seeds=True)

0 commit comments

Comments
 (0)