dag_processing code needs to handle OSError("handle is closed") in poll() and recv() calls
See original GitHub issueApache Airflow version
2.1.4
What happened
The problem also exists in the latest version of the Airflow code, but I experienced it in 2.1.4.
This is the root cause of problems experienced in issue#13542.
I’ll provide a stack trace below. The problem is in the code of airflow/dag_processing/processor.py (and manager.py), all poll() and recv() calls to the multiprocessing communication channels need to be wrapped in exception handlers, handling OSError(“handle is closed”) exceptions. If one looks at the Python multiprocessing source code, it throws this exception when the channel’s handle has been closed.
This occurs in Airflow when a DAG File Processor has been killed or terminated; the Airflow code closes the communication channel when it is killing or terminating a DAG File Processor process (for example, when a dag_file_processor_timeout occurs).This killing or terminating happens asynchronously (in another process) from the process calling the poll() or recv() on the communication channel. This is why an exception needs to be handled. A pre-check of the handle being open is not good enough, because the other process doing the kill or terminate may close the handle in between your pre-check and actually calling poll() or recv() (a race condition).
What you expected to happen
Here is the stack trace of the occurence I saw:
[2022-03-08 17:41:06,101] {taskinstance.py:914} DEBUG - <TaskInstance: staq_report_daily.gs.wait_staq_csv_file 2022-03-06 17:15:00+00:00 [running]> dependency 'Not In Retry Period' PASSED: True, The context specified that being in a retry p
eriod was permitted.
[2022-03-08 17:41:06,101] {taskinstance.py:904} DEBUG - Dependencies all met for <TaskInstance: staq_report_daily.gs.wait_staq_csv_file 2022-03-06 17:15:00+00:00 [running]>
[2022-03-08 17:41:06,119] {scheduler_job.py:1196} DEBUG - Skipping SLA check for <DAG: gdai_gcs_sync> because no tasks in DAG have SLAs
[2022-03-08 17:41:06,119] {scheduler_job.py:1196} DEBUG - Skipping SLA check for <DAG: unity_creative_import_process> because no tasks in DAG have SLAs
[2022-03-08 17:41:06,119] {scheduler_job.py:1196} DEBUG - Skipping SLA check for <DAG: sales_dm_to_bq> because no tasks in DAG have SLAs
[2022-03-08 17:44:50,454] {settings.py:302} DEBUG - Disposing DB connection pool (PID 1902)
Process ForkProcess-1:
Traceback (most recent call last):
File "/opt/python3.8/lib/python3.8/multiprocessing/process.py", line 315, in _bootstrap
self.run()
File "/opt/python3.8/lib/python3.8/multiprocessing/process.py", line 108, in run
self._target(*self._args, **self._kwargs)
File "/opt/python3.8/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 370, in _run_processor_manager
processor_manager.start()
File "/opt/python3.8/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 610, in start
return self._run_parsing_loop()
File "/opt/python3.8/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 671, in _run_parsing_loop
self._collect_results_from_processor(processor)
File "/opt/python3.8/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 981, in _collect_results_from_processor
if processor.result is not None:
File "/opt/python3.8/lib/python3.8/site-packages/airflow/dag_processing/processor.py", line 321, in result
if not self.done:
File "/opt/python3.8/lib/python3.8/site-packages/airflow/dag_processing/processor.py", line 286, in done
if self._parent_channel.poll():
File "/opt/python3.8/lib/python3.8/multiprocessing/connection.py", line 255, in poll
self._check_closed()
File "/opt/python3.8/lib/python3.8/multiprocessing/connection.py", line 136, in _check_closed
raise OSError("handle is closed")
OSError: handle is closed
This corresponded in time to the following log entries:
% kubectl logs airflow-scheduler-58c997dd98-n8xr8 -c airflow-scheduler --previous | egrep 'Ran scheduling loop in|[[]heartbeat[]]'
[2022-03-08 17:40:47,586] {scheduler_job.py:813} DEBUG - Ran scheduling loop in 0.56 seconds
[2022-03-08 17:40:49,146] {scheduler_job.py:813} DEBUG - Ran scheduling loop in 0.56 seconds
[2022-03-08 17:40:50,675] {base_job.py:227} DEBUG - [heartbeat]
[2022-03-08 17:40:50,687] {scheduler_job.py:813} DEBUG - Ran scheduling loop in 0.54 seconds
[2022-03-08 17:40:52,144] {scheduler_job.py:813} DEBUG - Ran scheduling loop in 0.46 seconds
[2022-03-08 17:40:53,620] {scheduler_job.py:813} DEBUG - Ran scheduling loop in 0.47 seconds
[2022-03-08 17:40:55,085] {scheduler_job.py:813} DEBUG - Ran scheduling loop in 0.46 seconds
[2022-03-08 17:40:56,169] {base_job.py:227} DEBUG - [heartbeat]
[2022-03-08 17:40:56,180] {scheduler_job.py:813} DEBUG - Ran scheduling loop in 0.49 seconds
[2022-03-08 17:40:57,667] {scheduler_job.py:813} DEBUG - Ran scheduling loop in 0.49 seconds
[2022-03-08 17:40:59,148] {scheduler_job.py:813} DEBUG - Ran scheduling loop in 0.48 seconds
[2022-03-08 17:41:00,618] {scheduler_job.py:813} DEBUG - Ran scheduling loop in 0.47 seconds
[2022-03-08 17:41:01,742] {base_job.py:227} DEBUG - [heartbeat]
[2022-03-08 17:41:01,757] {scheduler_job.py:813} DEBUG - Ran scheduling loop in 0.58 seconds
[2022-03-08 17:41:03,133] {scheduler_job.py:813} DEBUG - Ran scheduling loop in 0.55 seconds
[2022-03-08 17:41:04,664] {scheduler_job.py:813} DEBUG - Ran scheduling loop in 0.53 seconds
[2022-03-08 17:44:50,649] {base_job.py:227} DEBUG - [heartbeat]
[2022-03-08 17:44:50,814] {scheduler_job.py:813} DEBUG - Ran scheduling loop in 225.15 seconds
You can see that when this exception occurred, there was a hang in the scheduler for almost 4 minutes, no scheduling loops, and no scheduler_job heartbeats.
This hang probably also caused stuck queued jobs as issue#13542 describes.
How to reproduce
This is hard to reproduce because it is a race condition. But you might be able to reproduce by having in a dagfile top-level code that calls sleep, so that it takes longer to parse than core dag_file_processor_timeout setting. That would cause the parsing processes to be terminated, creating the conditions for this bug to occur.
Operating System
NAME=“Ubuntu” VERSION=“18.04.6 LTS (Bionic Beaver)” ID=ubuntu ID_LIKE=debian PRETTY_NAME=“Ubuntu 18.04.6 LTS” VERSION_ID=“18.04” HOME_URL=“https://www.ubuntu.com/” SUPPORT_URL=“https://help.ubuntu.com/” BUG_REPORT_URL=“https://bugs.launchpad.net/ubuntu/” PRIVACY_POLICY_URL=“https://www.ubuntu.com/legal/terms-and-policies/privacy-policy” VERSION_CODENAME=bionic UBUNTU_CODENAME=bionic
Versions of Apache Airflow Providers
Not relevant, this is a core dag_processing issue.
Deployment
Composer
Deployment details
“composer-1.17.6-airflow-2.1.4”
In order to isolate the scheduler to a separate machine, so as to not have interference from other processes such as airflow-workers running on the same machine, we created an additional node-pool for the scheduler, and ran these k8s patches to move the scheduler to a separate machine.
New node pool definition:
{
name = "scheduler-pool"
machine_type = "n1-highcpu-8"
autoscaling = false
node_count = 1
disk_type = "pd-balanced"
disk_size = 64
image_type = "COS"
auto_repair = true
auto_upgrade = true
max_pods_per_node = 32
},
patch.sh
#!/bin/bash
if [ $# -lt 1 ]; then
echo "Usage: $0 namespace"
echo "Description: Isolate airflow-scheduler onto it's own node-pool (scheduler-pool)."
echo "Options:"
echo " namespace: kubernetes namespace used by Composer"
exit 1
fi
namespace=$1
set -eu
set -o pipefail
scheduler_patch="$(cat airflow-scheduler-patch.yaml)"
fluentd_patch="$(cat composer-fluentd-daemon-patch.yaml)"
set -x
kubectl -n default patch daemonset composer-fluentd-daemon -p "${fluentd_patch}"
kubectl -n ${namespace} patch deployment airflow-scheduler -p "${scheduler_patch}"
composer-fluentd-daemon-patch.yaml
spec:
template:
spec:
nodeSelector: null
affinity:
nodeAffinity:
requiredDuringSchedulingIgnoredDuringExecution:
nodeSelectorTerms:
- matchExpressions:
- key: cloud.google.com/gke-nodepool
operator: In
values:
- default-pool
- scheduler-pool
airflow-scheduler-patch.yaml
spec:
template:
spec:
nodeSelector:
cloud.google.com/gke-nodepool: scheduler-pool
containers:
- name: gcs-syncd
resources:
limits:
memory: 2Gi
Anything else
On the below checkbox of submitting a PR, I could submit one, but it’d be untested code, I don’t really have the environment setup to test the patch.
Are you willing to submit PR?
- Yes I am willing to submit a PR!
Code of Conduct
- I agree to follow this project’s Code of Conduct
Issue Analytics
- State:
- Created 2 years ago
- Comments:11 (8 by maintainers)

Top Related StackOverflow Question
Trying to explain things…
Our team has run into this issue time and time again. We have tried different combinations of both Airflow and Python versions to no avail.
TL;DR
When a
DAGFileProcessorhangs and is killed due to a timeout we believe theself.waitablesandself._processorsattributes of theDAGFileProcessorManagerare not being updated as they should. This causes an unhandled exception when trying to receive data on a pipe end (i.e. file descriptor) which has already been closed.The long read…
We’re running a decouple Airflow deployment within a k8s cluster. We are currently using a 3-container pod where one of them runs the Web Server, another one executes the Scheduler and the third one implements Flower (we’re using the CeleryExecutor). The backbone of the deployment is implemented through a StatefulSet that runs the Celery executors themselves.
The trace we were seeing on the scheduler time and time again was:
This has been thrown by Airflow 2.1.3, but we’ve seen very similar (if not equal) variations with versions all the way up to Airflow 2.2.4.
Given we traced the problem down to the way multiprocessing synchronisation was being handled we played around with
multiprocessing’s start method through themp_start_methodconfiguration parameter which wasn’t included in the stock configuration example:https://github.com/apache/airflow/blob/f309ea78f7d8b62383bc41eac217681a0916382b/airflow/utils/mixins.py#L27-L38
The containers we are using leverage
forkas the default way of creating new process. After trying that one out we moved on to usingspawnand ended up settling onforkserver. No matter the start method we leveraged, we ran into the same issue over and over again.For a while we coped with this behaviour by just restarting the Airflow deployment on an hourly basis, but we decided to set some time apart today to delve a bit deeper into all this. The good news is after a thorough investigation we noticed a pattern the preceded the crash.
In order to pin it down down ran
ps(1)on the scheduler container. We also monitored the DAG Processor Manager log (which we have at/opt/airflow/logs/dag_processor_manager/dag_processor_manager.loggiven our Airflow home is/opt/airflow) and we took a look at the scheduler’s log throughkubectl logsgiven it’s sent to stdout/stderr. The pattern itself goes something like:DAGFileProcessorget’s stuck for longer thandag_file_processor_timeoutas seen onps’ output.DAGFileProcessorManagerkills the stuckDAGFileProcessor.DAGFileProcessorManagertries to collect results back from the differentDAGFileProcessors it crashes.The above led us to believe something was a bit off in the way the
DAGFileProcessors were being killed. Given our Docker-based deployment allowed for it, we retrieved a copy of the stockmanager.pyandprocessor.pyfiles and added a bit of logging throughself.log.debug(). The following appeared on outDAGFileProcessorManagerlog:Can you see how the number of
waitables(more on that later) doesn’t change even though we’re killing aDAGFileProcessor? We believe that’s what’s causing the trouble…Note we added the
- FOO -token to the logging entries we added to easilygrepfor them. These entries were generated with calls toself.log.debug()within the_kill_timed_out_processors()function. The ‘stock’ version is:https://github.com/apache/airflow/blob/614858fb7d443880451e6111b27fdaf942f563a4/airflow/dag_processing/manager.py#L1159-L1175
After we added additional logging it looked like:
You can see how we call the
kill()(which basically wraps the processor’s_kill_process()method) method on the timed out processor. We believe the key of all this resides online 246of:https://github.com/apache/airflow/blob/614858fb7d443880451e6111b27fdaf942f563a4/airflow/dag_processing/processor.py#L238-L246
Notice how the end of the communicating pipe opened on
line 187(below) is being closed on that line!https://github.com/apache/airflow/blob/614858fb7d443880451e6111b27fdaf942f563a4/airflow/dag_processing/processor.py#L187
That’s exactly the same pipe end (i.e. file descriptor) the
DAGFileProcessorManagertries to read from later on! If we look at the traceback we included before it’sline 286which ultimately triggers theOSError:https://github.com/apache/airflow/blob/614858fb7d443880451e6111b27fdaf942f563a4/airflow/dag_processing/processor.py#L286
What it’s trying to do is
poll()a closed pipe. If we take a look atmultiprocessing’s implementation we’ll check how, as shown on the traceback, it calls_check_closed()on the pipe’s file descriptor (i.e. handle) before proceeding: this, as seen before, triggers theOSError.So… why are we trying to collect results from a
DAGFileProcessorwe killed due to a timeout? in order to answer that we took a walk around_run_parsing_loop():https://github.com/apache/airflow/blob/614858fb7d443880451e6111b27fdaf942f563a4/airflow/dag_processing/manager.py#L612-L734
It basically runs an infinite (unless we specify a maximum number of runs) loop that calls
multiprocessing.connection.waitbased on the contents ofself.waitables. This attribute is a dictionary containing a reference to the differentDAGFileProcessor’s spawned by theDAGFileProcessorManager. Entries are added online 1034withinstart_new_process():https://github.com/apache/airflow/blob/614858fb7d443880451e6111b27fdaf942f563a4/airflow/dag_processing/manager.py#L1015-L1034
However, this dictionary is not updated when a processor is killed due to a timeout. You can check that out on the snippet we included above. Thus, after the timed out
DAGFileProcessoris killed, the infinite loop on_run_parsing_loop()will assume the underlying process is ready (it’s done, as we’ve effectively terminated it) and will try to read from the pipe end we closed on_kill_process(), thus triggering the exception and bringing everything down. In other words, we believe theself.waitablesattribute is not being updated as it should whenDAGFileProcessors are terminated due to timeouts. The same is true for theself._processorsattribute on theDAGFileProcessorManager. After all,_kill_timed_out_processors()iterates over its contents… If we don’t update it too we’ll see how we try to kill an already terminated process over and over again.After some testing we arrived at the following implementation of
_kill_timed_out_processors():We know the above can surely be written in a more succinct/better way: we’re by no means good programmers!
Against all odds, the code above seems to prevent the crash! 🎉 It does, however, spawn zombies when we kill the
DAGFileProcessor: it’s not beingwait()ed for…We decided to also play around with the
DAGFileProcessor’s_kill_process()method a bit in the name of science to try and prevent that zombie from spawning:From what we could see, the above reaped the zombie like we initially expected it to.
So, after all this nonsense we just wanted to end up by saying that we believe it’s the way
DAGFileManagerProcessor’s attributes are being cleaned up that crashes Airflow for us. In our experience this is triggered by aDAGFileProcessorbeing forcefully terminated after a timeout.We would also like to thank everybody making Airflow possible: it’s one heck of a tool!
Feel free to ask for more details and, if we got anything wrong (it wouldn’t be the first time), please do let us know!
Hi @potiuk! I’m glad you found our investigation useful and that you had fun reading through it. Reading so may Agatha Christie books has to pay off at some point 😜
I would be more than happy to polish it all up and open a Pull Request so that the changes are incorporated into Airflow itself. I’ll do my best to find some time to do it throughout the week.
And thanks a ton for the kind words! I really appreciate it 😋