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

Fix case where Executors fail to report failure, creating infinite loop #1220

Merged
merged 2 commits into from
Mar 28, 2016

Conversation

jlowin
Copy link
Member

@jlowin jlowin commented Mar 26, 2016

Finally tracked this one down! Closes #1199

When tasks run, they try to catch any errors and, if there are any, mark themselves failed. In addition,
SequentialExecutor and LocalExecutor both use subprocess.Popen to run Airflow commands, then wait() for the command to return. If an exception is raised, they mark the task as failed; otherwise they mark it as a success.

The problem here is that Popen doesn't actually raise errors -- you have to check the returncode explicitly. That means that Executors always return 'SUCCESS' for their commands (even when the command fails).

This is usually not an issue because a task's own status takes precedence over the executor's status; if the task reports failure and the executor reports success, failure wins. However, sometimes this IS a problem (as described in #1199) when an error is raised before the task even has a chance to run. In that case, the task reports no status and the executor reports SUCCESS. There is a trap for this case in jobs.py but unfortunately it leads to an infinite loop because the task is simply rescheduled.

This PR:

  1. modifies SequentialExecutor and LocalExecutor to use subprocess.check_call() instead of subprocess.Popen().wait(). Check_call raises CalledProcessError if the command returncode is not 0 OR if there is a genuine error calling the function.
  2. Checks for the case when TI says it's running but executor says it failed (as described by @mistercrunch here
  3. In case all else fails, monitors the "Unusual circumstances" block of jobs.py to see if an airflow run command failed more than 3 times. If it does, it intervenes and marks the task as failed. This prevents the infinite loop.

This DAG demonstrates the issue (inspired by #1168). It (illegally) changes the name of its subdag, which means the subdag can't be found and an error is raised before any tasks run. In the current Airflow master, this DAG loops forever.

from datetime import datetime

from airflow.models import DAG
from airflow.operators import SubDagOperator
from airflow.example_dags.subdags.subdag import subdag

args = {
    'owner': 'airflow',
    'start_date': datetime(2016, 1, 1),
}

dag = DAG(
    dag_id='test_raise_subdag_error',
    default_args=args,
    schedule_interval="@daily",
)

section_1 = SubDagOperator(
    task_id='subdag_op',
    subdag=subdag('test_raise_subdag_error', 'subdag_op', args),
    default_args=args,
    dag=dag,
)

# change the subdag name -- this creates an error because the subdag
# won't be found, but it'll do it in a way that causes the executor to report
# success
section_1.subdag.dag_id = 'bad_id'

Run it: airflow backfill -s 2016-01-01 -e 2016-01-01

To support this, I have written a "light" dag unit tester but it is not part of this PR.

@bolkedebruin
Copy link
Contributor

That's excellent! Would you mind squashing your commits into one? Please follow these guidelines for the commit message http://chris.beams.io/posts/git-commit. It helps creating changelogs and release notes in the future. Thanks!

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
@jlowin jlowin force-pushed the fix_executor_failed branch from 4026bbb to fc23b46 Compare March 26, 2016 13:36
@jlowin
Copy link
Member Author

jlowin commented Mar 26, 2016

@bolkedebruin thanks for the tip!

@landscape-bot
Copy link

Code Health
Repository health decreased by 0.01% when pulling fc23b46 on jlowin:fix_executor_failed into 7a632e2 on airbnb:master.

@coveralls
Copy link

Coverage Status

Coverage decreased (-0.09%) to 66.288% when pulling fc23b46 on jlowin:fix_executor_failed into 7a632e2 on airbnb:master.

@landscape-bot
Copy link

Code Health
Repository health increased by 0.10% when pulling f0eeb15 on jlowin:fix_executor_failed into 7a632e2 on airbnb:master.

@coveralls
Copy link

Coverage Status

Coverage increased (+0.07%) to 66.448% when pulling f0eeb15 on jlowin:fix_executor_failed into 7a632e2 on airbnb:master.

@bolkedebruin bolkedebruin merged commit d7c95a8 into apache:master Mar 28, 2016
@jlowin jlowin deleted the fix_executor_failed branch March 29, 2016 13:21
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

Successfully merging this pull request may close these issues.

4 participants