[ 
https://issues.apache.org/jira/browse/AIRFLOW-4297?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16816354#comment-16816354
 ] 

Xiaodong DENG commented on AIRFLOW-4297:
----------------------------------------

dag.following_schedule will only return None if schedule_interval is None or 
"@once".

In this case, manage_slas() will throw exception as None can't be compared with 
datetime.utcnow().

 

The root cause is that manage_slas() didn't consider 
schedule_interval=None/"@once". The logic 
{code:java}
if task.sla:
        dttm = dag.following_schedule(dttm)
        while dttm < datetime.utcnow():
        following_schedule = dag.following_schedule(dttm)
        if following_schedule + task.sla < datetime.utcnow():
        session.merge(models.SlaMiss(
        task_id=ti.task_id,
        dag_id=ti.dag_id,
        execution_date=dttm,
        timestamp=ts))
{code}
is designed purely for DAGs with scheduler_interval !=None/"@once".

> Manually triggerd DAG with no schedule_interval breaks scheduler
> ----------------------------------------------------------------
>
>                 Key: AIRFLOW-4297
>                 URL: https://issues.apache.org/jira/browse/AIRFLOW-4297
>             Project: Apache Airflow
>          Issue Type: Bug
>    Affects Versions: 1.10.3
>            Reporter: Ash Berlin-Taylor
>            Priority: Blocker
>             Fix For: 1.10.4
>
>
> {code:title=example_dag.py}
> timezone = "UTC"
> local_tz = pendulum.timezone(timezone)
> start_date = datetime.datetime.strptime('2019-03-28 07:57:00', "%Y-%m-%d 
> %H:%M:%S")
> start_date = start_date.replace(tzinfo=local_tz)
> default_args = {
> 'owner': 'DaniRC'
> }
> dag = DAG('testsla',
>     default_args=default_args,
>     start_date=start_date,
>     concurrency=1,
>     max_active_runs=1,
>     default_view='tree',
>     orientation='TB',
>     catchup=False,
>     schedule_interval=None
>     )
> {code}
> If this DAG is triggered, then this error occurs:
> {noformat}
> Process DagFileProcessor5303-Process:
> Traceback (most recent call last):
>   File "/usr/lib/python3.6/multiprocessing/process.py", line 258, in 
> _bootstrap
>     self.run()
>   File "/usr/lib/python3.6/multiprocessing/process.py", line 93, in run
>     self._target(*self._args, **self._kwargs)
>   File "/usr/lib/python3.6/site-packages/airflow/jobs.py", line 402, in helper
>     pickle_dags)
>   File "/usr/lib/python3.6/site-packages/airflow/utils/db.py", line 73, in 
> wrapper
>     return func(*args, **kwargs)
>   File "/usr/lib/python3.6/site-packages/airflow/jobs.py", line 1760, in 
> process_file
>     self._process_dags(dagbag, dags, ti_keys_to_schedule)
>   File "/usr/lib/python3.6/site-packages/airflow/jobs.py", line 1452, in 
> _process_dags
>     self.manage_slas(dag)
>   File "/usr/lib/python3.6/site-packages/airflow/utils/db.py", line 73, in 
> wrapper
>     return func(*args, **kwargs)
>   File "/usr/lib/python3.6/site-packages/airflow/jobs.py", line 662, in 
> manage_slas
>     while dttm < timezone.utcnow():
> TypeError: '<' not supported between instances of 'NoneType' and 
> 'datetime.datetime'
> {noformat}
> After this happens the scheduler also won't schedule any further tasks for 
> _ANY_ dag.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to