Skip to content

Bad file descriptor in task runner in case of missing DAG file #55787

@kosteev

Description

@kosteev

Apache Airflow version

3.0.6

If "Other Airflow 2 version" selected, which one?

No response

What happened?

In Airflow 3 I noticed an issue with task runner improperly behaving in case of missing DAG during parsing, basically when reaching this place:

Image

After debugging, I figure out that calling "exit" somehow abnormally closes supervisor-runner descriptor and further communication between is broken.
I found that, if to replace exit -> sys.exit, then it works fine.

According to https://pavolkutaj.medium.com/explaining-the-difference-between-exiting-functions-with-exit-and-raise-in-python-ceda89ef2846

  • exit() is intended for use in the interactive shell.
  • sys.exit() is intended for use in programs.
  • sys.exit() is recommended for use in production code.

It looks like indeed builtin exit is closing stdin descriptor (which is supervisor socket), as this is its implementation:

class Quitter(object):
    def __init__(self, name, eof):
        self.name = name
        self.eof = eof
    def __repr__(self):
        return 'Use %s() or %s to exit' % (self.name, self.eof)
    def __call__(self, code=None):
        # Shells like IDLE catch the SystemExit, but listen when their
        # stdin wrapper is closed.
        try:
            sys.stdin.close()
        except:
            pass
        raise SystemExit(code)

What you think should happen instead?

No response

How to reproduce

Remove DAG file during execution

Operating System

Linux

Versions of Apache Airflow Providers

No response

Deployment

Official Apache Airflow Helm Chart

Deployment details

No response

Anything else?

No response

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

Metadata

Metadata

Assignees

No one assigned

    Labels

    area:corekind:bugThis is a clearly a bugneeds-triagelabel for new issues that we didn't triage yet

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions