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

Adding an example to illustrate the TriggerDagRunOperator #1043

Merged
merged 4 commits into from
Feb 20, 2016
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
48 changes: 48 additions & 0 deletions airflow/example_dags/example_trigger_controller_dag.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
from airflow import DAG, utils
from airflow.operators import *
from datetime import date, datetime, time, timedelta

import pprint

pp = pprint.PrettyPrinter(indent=4)

# This example illustrates the use of the TriggerDagRunOperator. There are 2
# entities at work in this scenario:
# 1. The Controller DAG - the DAG that conditionally executes the trigger
# 2. The Target DAG - DAG being triggered (in example_trigger_target_dag.py)
#
# This example illustrates the following features :
# 1. A TriggerDagRunOperator that takes:
# a. A python callable that decides whether or not to trigger the Target DAG
# b. An optional params dict passed to the python callable to help in
# evaluating whether or not to trigger the Target DAG
# c. The id (name) of the Target DAG
# d. The python callable can add contextual info to the DagRun created by
# way of adding a Pickleable payload (e.g. dictionary of primitives). This
# state is then made available to the TargetDag
# 2. A Target DAG : c.f. example_trigger_target_dag.py

# This function decides whether or not to Trigger the remote DAG
def conditionally_trigger(context, dag_run_obj):
c_p =context['params']['condition_param']
print("Controller DAG : conditionally_trigger = {}".format(c_p))
if context['params']['condition_param']:
dag_run_obj.payload = {'message' :context['params']['message'] }
pp.pprint(dag_run_obj.payload)
return dag_run_obj


# Define the DAG
dag = DAG(dag_id='example_trigger_controller_dag',
default_args={"owner" : "me",
"start_date":datetime.now()},
schedule_interval='@once')


# Define the single task in this controller example DAG
trigger = TriggerDagRunOperator(task_id='test_trigger_dagrun',
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

need import TriggerDagRunOperator

trigger_dag_id="example_trigger_target_dag",
python_callable=conditionally_trigger,
params={'condition_param':True,
'message':'Hello World'},
dag=dag)
43 changes: 43 additions & 0 deletions airflow/example_dags/example_trigger_target_dag.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
from airflow.operators import *
from airflow.models import DAG
from datetime import date, datetime, time, timedelta

import pprint
pp = pprint.PrettyPrinter(indent=4)

# This example illustrates the use of the TriggerDagRunOperator. There are 2
# entities at work in this scenario:
# 1. The Controller DAG - the DAG that conditionally executes the trigger
# (in example_trigger_controller.py)
# 2. The Target DAG - DAG being triggered
#
# This example illustrates the following features :
# 1. A TriggerDagRunOperator that takes:
# a. A python callable that decides whether or not to trigger the Target DAG
# b. An optional params dict passed to the python callable to help in
# evaluating whether or not to trigger the Target DAG
# c. The id (name) of the Target DAG
# d. The python callable can add contextual info to the DagRun created by
# way of adding a Pickleable payload (e.g. dictionary of primitives). This
# state is then made available to the TargetDag
# 2. A Target DAG : c.f. example_trigger_target_dag.py

args = {
'start_date': datetime.now(),
'owner': 'airflow',
}

dag = DAG(
dag_id='example_trigger_target_dag',
default_args=args,
schedule_interval=None)


def run_this_func(ds, **kwargs):
print( "Remotely received value of {} for key=message".format(kwargs['dag_run'].conf['message']))

run_this = PythonOperator(
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

don't you need import PythonOperator?

task_id='run_this',
provide_context=True,
python_callable=run_this_func,
dag=dag)
2 changes: 1 addition & 1 deletion tests/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
from airflow.utils import AirflowException
from airflow.configuration import AirflowConfigException

NUM_EXAMPLE_DAGS = 7
NUM_EXAMPLE_DAGS = 9
DEV_NULL = '/dev/null'
DEFAULT_DATE = datetime(2015, 1, 1)
DEFAULT_DATE_ISO = DEFAULT_DATE.isoformat()
Expand Down