SQL_ALCHEMY_CONN_CMD causes triggerers to fail liveness probes on peak #33485
Labels
area:core
kind:bug
This is a clearly a bug
needs-triage
label for new issues that we didn't triage yet
Apache Airflow version
Other Airflow 2 version (please specify below)
What happened
Airflow version: 2.5.3
Related to this comment from @vchiapaikeo: #33172 (comment)
A couple of mins after midnight UTC - when 100s of DAGs are kicked off - we noticed our triggerer replicas failing liveness probe checks and restarting systematically.
Further profiling led to the discovery that the triggerer’s sync loop hangs for several minutes when there are 1000s of triggers running simultaneously, specifically while bulk fetching triggers, which causes the triggerer to miss heartbeats and eventually get restarted by k8s.
With profiling still enabled, we observed that while the trigger is hanging and we profile the execution, we get this stack trace:
Which indicates that airflow is running a subprocess for each fetched row and that takes the vast majority of the execution time.
We found that during the unmarshaling of the resulting rows into the Trigger model, the kwargs column (ExtendedJSON) runs process_returned_value, on each row, and reads the
SQL_ALCHEMY_CONN
configuration to determine whether the engine supports json or not and parse kwargs accordingly. However, in our case we defineSQL_ALCHEMY_CONN_CMD
as opposed toSQL_ALCHEMY_CONN
, which causes the sync loop to spawn a new subprocess for every row (here).We workaround it by using
SQL_ALCHEMY_CONN
instead ofSQL_ALCHEMY_CONN_CMD
, as it involves reading an environment variable instead of spawning a new subprocess.What you think should happen instead
The triggerer model caches caches either the
SQL_ALCHEMY_CONN
or the db_supports_json property.How to reproduce
Simultaneously kick off 100s of DAGs with at least a few deferrable operators each and use
SQL_ALCHEMY_CONN_CMD
instead ofSQL_ALCHEMY_CONN
Operating System
Debian GNU/Linux 11 (bullseye)
Versions of Apache Airflow Providers
apache-airflow-providers-airbyte==3.2.0
apache-airflow-providers-alibaba==2.2.0
apache-airflow-providers-amazon==7.3.0
apache-airflow-providers-apache-beam==4.3.0
apache-airflow-providers-apache-cassandra==3.1.1
apache-airflow-providers-apache-drill==2.3.1
apache-airflow-providers-apache-druid==3.3.1
apache-airflow-providers-apache-hdfs==3.2.0
apache-airflow-providers-apache-hive==5.1.3
apache-airflow-providers-apache-kylin==3.1.0
apache-airflow-providers-apache-livy==3.3.0
apache-airflow-providers-apache-pig==4.0.0
apache-airflow-providers-apache-pinot==4.0.1
apache-airflow-providers-apache-spark==4.0.0
apache-airflow-providers-apache-sqoop==3.1.1
apache-airflow-providers-arangodb==2.1.1
apache-airflow-providers-asana==2.1.0
apache-airflow-providers-atlassian-jira==2.0.1
apache-airflow-providers-celery==3.1.0
apache-airflow-providers-cloudant==3.1.0
apache-airflow-providers-cncf-kubernetes==5.2.2
apache-airflow-providers-common-sql==1.3.4
apache-airflow-providers-databricks==4.0.0
apache-airflow-providers-datadog==3.1.0
apache-airflow-providers-dbt-cloud==3.1.0
apache-airflow-providers-dingding==3.1.0
apache-airflow-providers-discord==3.1.0
apache-airflow-providers-docker==3.5.1
apache-airflow-providers-elasticsearch==4.4.0
apache-airflow-providers-exasol==4.1.3
apache-airflow-providers-facebook==3.1.0
apache-airflow-providers-ftp==3.3.1
apache-airflow-providers-github==2.2.1
apache-airflow-providers-google==8.11.0
apache-airflow-providers-grpc==3.1.0
apache-airflow-providers-hashicorp==3.3.0
apache-airflow-providers-http==4.2.0
apache-airflow-providers-imap==3.1.1
apache-airflow-providers-influxdb==2.1.0
apache-airflow-providers-jdbc==3.3.0
apache-airflow-providers-jenkins==3.2.0
apache-airflow-providers-microsoft-azure==5.2.1
apache-airflow-providers-microsoft-mssql==3.3.2
apache-airflow-providers-microsoft-psrp==2.2.0
apache-airflow-providers-microsoft-winrm==3.1.1
apache-airflow-providers-mongo==3.1.1
apache-airflow-providers-mysql==4.0.2
apache-airflow-providers-neo4j==3.2.1
apache-airflow-providers-odbc==3.2.1
apache-airflow-providers-openfaas==3.1.0
apache-airflow-providers-opsgenie==5.0.0
apache-airflow-providers-oracle==3.6.0
apache-airflow-providers-pagerduty==3.1.0
apache-airflow-providers-papermill==3.1.1
apache-airflow-providers-plexus==3.1.0
apache-airflow-providers-postgres==5.4.0
apache-airflow-providers-presto==4.2.2
apache-airflow-providers-qubole==3.3.1
apache-airflow-providers-redis==3.1.0
apache-airflow-providers-salesforce==5.3.0
apache-airflow-providers-samba==4.1.0
apache-airflow-providers-segment==3.1.0
apache-airflow-providers-sendgrid==3.1.0
apache-airflow-providers-sftp==4.2.4
apache-airflow-providers-singularity==3.1.0
apache-airflow-providers-slack==7.2.0
apache-airflow-providers-snowflake==4.0.4
apache-airflow-providers-sqlite==3.3.1
apache-airflow-providers-ssh==3.5.0
apache-airflow-providers-tableau==4.1.0
apache-airflow-providers-tabular==1.1.0
apache-airflow-providers-telegram==4.0.0
apache-airflow-providers-trino==4.3.2
apache-airflow-providers-vertica==3.3.1
apache-airflow-providers-yandex==3.3.0
apache-airflow-providers-zendesk==4.2.0
Deployment
Other 3rd-party Helm chart
Deployment details
Chart based on the official helm chart. Airflow running on Google Kubernetes Engine (GKE) using
KubernetesExecutor
.Anything else
No response
Are you willing to submit PR?
Code of Conduct
The text was updated successfully, but these errors were encountered: