-
Notifications
You must be signed in to change notification settings - Fork 14.4k
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
[AIRFLOW-56] Airflow's scheduler can "lose" queued tasks #1378
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -817,16 +817,25 @@ def error(self, session=None): | |
session.commit() | ||
|
||
@provide_session | ||
def refresh_from_db(self, session=None): | ||
def refresh_from_db(self, session=None, lock_for_update=False): | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. document lock_for_update |
||
""" | ||
Refreshes the task instance from the database based on the primary key | ||
|
||
:param lock_for_update: if True, indicates that the database should | ||
lock the TaskInstance (issuing a FOR UPDATE clause) until the session | ||
is committed. | ||
""" | ||
TI = TaskInstance | ||
ti = session.query(TI).filter( | ||
|
||
qry = session.query(TI).filter( | ||
TI.dag_id == self.dag_id, | ||
TI.task_id == self.task_id, | ||
TI.execution_date == self.execution_date, | ||
).first() | ||
TI.execution_date == self.execution_date) | ||
|
||
if lock_for_update: | ||
ti = qry.with_for_update().first() | ||
else: | ||
ti = qry.first() | ||
if ti: | ||
self.state = ti.state | ||
self.start_date = ti.start_date | ||
|
@@ -1159,7 +1168,7 @@ def run( | |
self.pool = pool or task.pool | ||
self.test_mode = test_mode | ||
self.force = force | ||
self.refresh_from_db() | ||
self.refresh_from_db(session=session, lock_for_update=True) | ||
self.clear_xcom_data() | ||
self.job_id = job_id | ||
iso = datetime.now().isoformat() | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,6 +23,8 @@ | |
from airflow.models import DAG | ||
from airflow.operators import DummyOperator, PythonOperator, SubDagOperator | ||
from airflow.utils.trigger_rule import TriggerRule | ||
import time | ||
|
||
DEFAULT_DATE = datetime(2016, 1, 1) | ||
default_args = dict( | ||
start_date=DEFAULT_DATE, | ||
|
@@ -31,6 +33,16 @@ | |
def fail(): | ||
raise ValueError('Expected failure.') | ||
|
||
def delayed_fail(): | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sleeping in tests is an antipattern, usually you want to want to use concurrency mechanisms instead. Can you put a TODO documenting this so we can fix it later? |
||
""" | ||
Delayed failure to make sure that processes are running before the error | ||
is raised. | ||
|
||
TODO handle more directly (without sleeping) | ||
""" | ||
time.sleep(5) | ||
raise ValueError('Expected failure.') | ||
|
||
# DAG tests backfill with pooled tasks | ||
# Previously backfill would queue the task but never run it | ||
dag1 = DAG(dag_id='test_backfill_pooled_task_dag', default_args=default_args) | ||
|
@@ -123,7 +135,9 @@ def fail(): | |
end_date=DEFAULT_DATE, | ||
default_args=default_args) | ||
dag8_task1 = PythonOperator( | ||
python_callable=fail, | ||
# use delayed_fail because otherwise LocalExecutor will have a chance to | ||
# complete the task | ||
python_callable=delayed_fail, | ||
task_id='test_queued_task', | ||
dag=dag8, | ||
pool='test_queued_pool') |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: Should log if the state is running