RikHeijdens opened a new issue #14384:
URL: https://github.com/apache/airflow/issues/14384


   **Apache Airflow version**: 2.0.1
   **Kubernetes version (if you are using kubernetes)** (use `kubectl 
version`): N/A
   **Environment**:
   
   - **Cloud provider or hardware configuration**:
   - **OS** (e.g. from /etc/os-release):
   - **Kernel** (e.g. `uname -a`):
   - **Install tools**:
   - **Others**: Python 3.8
   
   **What happened**:
   
   Occasionally, the Airflow scheduler crashes with the following exception:
   
   ```
   Exception when executing SchedulerJob._run_scheduler_loop
   
   Traceback (most recent call last):
     File 
"/usr/local/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 
1280, in _execute
       self._run_scheduler_loop()
     File 
"/usr/local/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 
1382, in _run_scheduler_loop
       num_queued_tis = self._do_scheduling(session)
     File 
"/usr/local/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 
1521, in _do_scheduling
       self._schedule_dag_run(dag_run, 
active_runs_by_dag_id.get(dag_run.dag_id, set()), session)
     File 
"/usr/local/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 
1760, in _schedule_dag_run
       schedulable_tis, callback_to_run = dag_run.update_state(session=session, 
execute_callbacks=False)
     File "/usr/local/lib/python3.8/site-packages/airflow/utils/session.py", 
line 62, in wrapper
       return func(*args, **kwargs)
     File "/usr/local/lib/python3.8/site-packages/airflow/models/dagrun.py", 
line 478, in update_state
       self._emit_duration_stats_for_finished_state()
     File "/usr/local/lib/python3.8/site-packages/airflow/models/dagrun.py", 
line 615, in _emit_duration_stats_for_finished_state
       duration = self.end_date - self.start_date
   TypeError: unsupported operand type(s) for -: 'datetime.datetime' and 
'NoneType'
   ```
   
   Just before this error surfaces, Airflow typically logs either one of the 
following messages:
   
   ```
   Marking run <DagRun sessionization @ 2021-02-15 08:05:00+00:00: 
scheduled__2021-02-15T08:05:00+00:00, externally triggered: False> failed
   ```
   
   or
   
   ```
   Marking run <DagRun sessionization @ 2021-02-16 08:05:00+00:00: 
scheduled__2021-02-16T08:05:00+00:00, externally triggered: False> successful
   ```
   
   The cause of this issue appears to be that the scheduler is attempting to 
update the state of a `DagRun` instance that is in a _running_ state, but does 
**not** have a `start_date` set. This will eventually cause a `TypeError` to be 
raised at L615 in `_emit_duration_stats_for_finished_state()` because `None` is 
subtracted from a `datetime` object.
   
   During my testing I was able to resolve the issue by manually updating any 
records in the `DagRun` table which are missing a `start_date`.
   
   However, it is a bit unclear to me _how_ it is possible for a DagRun 
instance to be transitioned into a `running` state, without having a 
`start_date` set. I spent some time digging through the code, and I believe the 
only code path that would allow a `DagRun` to end up in such a scenario is the 
state transition that occurs at L475 in 
[DagRun](https://github.com/apache/airflow/blob/2.0.1/airflow/models/dagrun.py#L475)
 where `DagRun.set_state(State.RUNNING)` is invoked without verifying that a 
`start_date` is set.
   
   **What you expected to happen**:
   
   I expect the Airflow scheduler not to crash, and to handle this scenario 
gracefully. 
   
   I have the impression that this is an edge-case, and even handling a missing 
`start_date` to be equal to a set `end_date` in 
`_emit_duration_stats_for_finished_state()` seems like a more favorable 
solution than raising a `TypeError` in the scheduler to me.
   
   **How to reproduce it**:
   
   I haven't been able to figure out a scenario which allows me to reproduce 
this issue reliably. We've had this issue surface for a fairly complex DAG 
twice in a time span of 5 days. We run over 25 DAGs on our Airflow instance, 
and so far the issue seems to be isolated to a single DAG.
   
   **Anything else we need to know**:
   
   While I'm unclear on what exactly causes a DagRun instance to not have a 
`start_date` set, the problem seems to be isolated to a single DAG on our 
Airflow instance. This DAG is fairly complex in the sense that it contains a 
number of root tasks, that have SubDagOperator's set as downstream (leaf) 
dependencies. These SubDagOperators each represent a SubDag containing between 
2 and 12 tasks. The SubDagOperator's have `depends_on_past` set to True, and 
`catchup` is enabled for the parent DAG. The parent DAG also has 
`max_active_runs` set to limit concurrency.
   
   I also have the impression, that this issue mainly surfaces when there are 
multiple DagRuns running concurrently for this DAG, but I don't have hard 
evidence for this. I did at some point clear task instance states, and 
transition the DagRun's state from `failed` back to `running` through the Web 
UI around the period that some of these issues arose.
   
   I've also been suspecting that this [block of 
code](https://github.com/apache/airflow/blob/2.0.1/airflow/jobs/scheduler_job.py#L1498-L1511)
 in `_do_scheduling` may be related to this issue, in the sense that I've been 
suspecting that there exists an edge case in which Task Instances may be 
considered active for a particular `execution_date`, but for which the DagRun 
object itself is not "active". My hypothesis is that this would eventually 
cause the "inactive" DagRun to be transitioned to `running` in 
`DagRun.set_state()` without ensuring that a `start_date` was set for the 
DagRun. I haven't been able to gather strong evidence for this hypothesis yet, 
though, and I'm hoping that someone more familiar with the implementation will 
be able to provide some guidance as to whether that hypothesis makes sense or 
not.
   
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to