-
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
Race condition between Triggerer and Scheduler #23824
Comments
One race condition has been fixed in #21316, but this is unrelated. |
Perhaps @ashb or @andrewgodwin have any ideas how do "synchronise" schedulers and triggereres to avoid this situation. |
I was able to replicate this issue with the following DAGs on my local machine with LocalExceutor. from airflow import DAG
from airflow.sensors.time_delta import TimeDeltaSensorAsync
from datetime import datetime, timedelta
for x in range(10):
_dag_id = f"sense_dag_{x}"
task_list = []
with DAG(
dag_id=_dag_id,
schedule_interval="*/1 * * * *",
start_date=datetime(2022, 1, 1),
max_active_runs=1,
# catchup=False,
) as dag:
for y in range(10):
task_list.append(
TimeDeltaSensorAsync(
task_id=f"sense_{y}",
delta=timedelta(seconds=0),
retries=3,
retry_delay=timedelta(seconds=5)
)
)
globals()[_dag_id] = dag This issue is made more clear when there are multiple retries because the task log that indicate a task cannot run is recorded in the next try as the try number is increased. Another clear indicator is the output from the scheduler_job.py where the task instance state is checked to be completed (not queued).
I believe the purpose of the check is to ensure that the task is not executed again before the scheduler acknowledges the event sent by the executor, which is only sent when a task execution completes whether it succeeded or failed. The check is to prevent tasks getting stuck in the queued state. This behaviour is outlined in AIRFLOW-1641 and fixed in #2715. With the introduction of Triggerers, there is a scenario where task aren't necessarily stuck in queued but placed in queued because it was moved to the scheduled state post completion of the trigger instance. This is roughly how the task instance state flows for deferred tasks.
However, given how long it takes for the scheduler to get to processing executor events in the scheduler loop, step 5 and 6 could be flipped. This means that the task instance is in the queued state when the scheduler processes the executor event of the deferred task instance. Since the state is queued, the condition is met; therefore the task is marked as failed / up for retry. |
I wonder if it's possible to have an in between state to indicate that the task has completed its execution but the scheduler has not processes the corresponding executor event in the event buffer. |
I'm seeing a possible regression (or at least a similar manifestation of this issue) in 2.4.3 using ExternalTaskSensorAsync |
@RNHTTR better to open a new issue with additional details and how to reproduce it. However it also could be a problem with As far as I know many of deferrable Operators use |
Apache Airflow version
2.2.5
What happened
Deferable tasks, that trigger instantly after getting defered, might get its state set to
FAILED
by the scheduler.The triggerer can fire the trigger and scheduler can re-queue the task instance before it has a chance to process the executor event for when the ti got defered.
What you think should happen instead
This code block should not run in this instance:
airflow/airflow/jobs/scheduler_job.py
Lines 667 to 692 in 5bfacf8
How to reproduce
Most importantly have a trigger, that instantly fires. I'm not sure if the executor type is important - I'm running
CeleryExecutor
. Also having two schedulers might be important.Operating System
Arch Linux
Versions of Apache Airflow Providers
No response
Deployment
Virtualenv installation
Deployment details
No response
Anything else
No response
Are you willing to submit PR?
Code of Conduct
The text was updated successfully, but these errors were encountered: