Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[AIRFLOW-3556] Add cross join set downstream function #4356

Merged
merged 1 commit into from
Dec 26, 2018
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
31 changes: 31 additions & 0 deletions airflow/utils/helpers.py
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,37 @@ def chain(*tasks):
up_task.set_downstream(down_task)


def cross_downstream(from_tasks, to_tasks):
"""
Set downstream dependencies for all tasks in from_tasks to all tasks in to_tasks.
E.g.: cross_downstream(from_tasks=[t1, t2, t3], to_tasks=[t4, t5, t6])
Is equivalent to:

t1 --> t4
\ /
t2 -X> t5
/ \
t3 --> t6

t1.set_downstream(t4)
t1.set_downstream(t5)
t1.set_downstream(t6)
t2.set_downstream(t4)
t2.set_downstream(t5)
t2.set_downstream(t6)
t3.set_downstream(t4)
t3.set_downstream(t5)
t3.set_downstream(t6)

:param from_tasks: List of tasks to start from.
:type from_tasks: List[airflow.models.BaseOperator]
:param to_tasks: List of tasks to set as downstream dependencies.
:type to_tasks: List[airflow.models.BaseOperator]
"""
for task in from_tasks:
task.set_downstream(to_tasks)


def pprinttable(rows):
"""Returns a pretty ascii table from tuples

Expand Down
17 changes: 16 additions & 1 deletion tests/utils/test_helpers.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,16 @@
import logging
import multiprocessing
import os
import psutil
import signal
import time
import unittest
from datetime import datetime

import psutil
import six

from airflow import DAG
from airflow.operators.dummy_operator import DummyOperator
from airflow.utils import helpers


Expand Down Expand Up @@ -210,6 +215,16 @@ def test_is_container(self):
# Pass an object that is not iter nor a string.
self.assertFalse(helpers.is_container(10))

def test_cross_downstream(self):
"""Test if all dependencies between tasks are all set correctly."""
dag = DAG(dag_id="test_dag", start_date=datetime.now())
start_tasks = [DummyOperator(task_id="t{i}".format(i=i), dag=dag) for i in range(1, 4)]
end_tasks = [DummyOperator(task_id="t{i}".format(i=i), dag=dag) for i in range(4, 7)]
helpers.cross_downstream(from_tasks=start_tasks, to_tasks=end_tasks)

for start_task in start_tasks:
six.assertCountEqual(self, start_task.get_direct_relatives(upstream=False), end_tasks)


if __name__ == '__main__':
unittest.main()