Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

BUG: AirflowException doesn't raise exception in SequentialExecutor or LocalExecutor #1199

Closed
jlowin opened this issue Mar 22, 2016 · 2 comments

Comments

@jlowin
Copy link
Member

jlowin commented Mar 22, 2016

Many bugs I've addressed recently share a symptom -- they all log an error that "The airflow run command failed at reporting an error. This should not occur in normal circumstances." (from https://github.com/airbnb/airflow/blob/master/airflow/jobs.py#L808)

Trying to reproduce. These are my notes...

Sequence of events:

  1. Use SequentialExecutor, as with airflow test ...

  2. The SequentialExecutor runs the task airflow command with (https://github.com/airbnb/airflow/blob/master/airflow/executors/sequential_executor.py):

    try:
        sp = subprocess.Popen(command, shell=True)
        sp.wait()
    except Exception as e:
        self.change_state(key, State.FAILED)
        raise e
    self.change_state(key, State.SUCCESS)
  3. A task raises AirflowException prior to being executing

  4. For some reason, the AirflowException is NOT caught by the executor's trap

    • pretty sure this is the problem, still exploring
    • don't we need to check sp.returncode rather than trapping errors?
  5. The state gets changed to success

  6. The task's state doesn't change

  7. The "normal circumstances" error gets raised

@mistercrunch
Copy link
Member

I think when there are kill -9 on a task instance you may get that as well.

@mistercrunch
Copy link
Member

There's an obvious elif missing before this message is logged, it's if the task instance state in the DB is running and the executor reported failure, that a clear failure. The handle_failure method should be called at that point and the failure recorded.

jlowin added a commit to jlowin/airflow that referenced this issue Mar 26, 2016
SequentialExecutor and LocalExecutor execute `airflow run` commands
with `subprocess.Popen().wait()` and try to catch errors with
`try/except`. However, `subprocess.Popen()` doesn't raise errors;
you have to check the `returncode`. As a result, these Executors
always report that their commands are successful. This is normally fine
because task status gets precedence over executor status, so as long
as the task reports on itself correctly the issue is avoided. But
if an error is raised BEFORE a task runs -- meaning the task is not
yet monitoring its own status -- then the executor will incorrectly
report success. Airflow will actually notice something went wrong,
but because the task doesn't say it failed, it gets rescheduled,
leading to an infinite loop.

To resolve this, replace the Executor's `Popen().wait()` with
`check_call()`, which is a blocking method that raises an error
if the returncode != 0. This way, errors are properly recognized.

Also, prevent infinite loops by limiting the number of times a
task is allowed to be rescheduled due to executor failure to 3.
(Note: this doesn't affect the number of times a task can be
rescheduled due to its own failure).

Last, check for an odd situation where the executor reports failure
but the task reports running.

Closes apache#1199
See apache#1220 for a test case
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

No branches or pull requests

2 participants