Skip to content

Commit

Permalink
a task needs to have start_date; either from the dag or the task
Browse files Browse the repository at this point in the history
  • Loading branch information
Krishna Puttaswamy committed Nov 22, 2014
1 parent 41ea65f commit 8b3bfe5
Show file tree
Hide file tree
Showing 3 changed files with 8 additions and 1 deletion.
6 changes: 6 additions & 0 deletions airflow/models.py
Original file line number Diff line number Diff line change
Expand Up @@ -937,6 +937,7 @@ def __init__(
utils.validate_key(dag_id)
self.dag_id = dag_id
self.executor = executor
self.start_date = start_date
self.end_date = end_date or datetime.now()
self.parallelism = parallelism
self.schedule_interval = schedule_interval
Expand Down Expand Up @@ -1087,6 +1088,11 @@ def get_downstream(task, level=0):
get_downstream(t)

def add_task(self, task):
if not self.start_date and not task.start_date:
raise Exception("Task is missing the start_date parameter")
if not task.start_date:
task.start_date = self.start_date

if task.task_id in [t.task_id for t in self.tasks]:
raise Exception("Task already added")
else:
Expand Down
2 changes: 1 addition & 1 deletion dags/examples/simple.py
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@
}

# Initializing a directed acyclic graph
dag = DAG(dag_id='simple', executor=LocalExecutor())
dag = DAG(dag_id='simple', executor=LocalExecutor(), start_date=datetime(2014, 10, 10))

# MySQL Operator
sql = "TRUNCATE TABLE tmp;"
Expand Down
1 change: 1 addition & 0 deletions dags/examples/test_wf_external.py
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@
# Setting some default operator parameters
default_args = {
'owner': 'max',
'start_date': datetime(2014, 10, 20),
'mysql_dbid': 'local_mysql',
}

Expand Down

0 comments on commit 8b3bfe5

Please sign in to comment.