zambadruzaman opened a new issue #21059:
URL: https://github.com/apache/airflow/issues/21059


   ### Apache Airflow version
   
   2.2.2
   
   ### What happened
   
   Dag dependency view is not rendering for Postgres backed Airflow.
   
   ### What you expected to happen
   
   In this [if 
statements](https://github.com/apache/airflow/blob/cecd4c8059e04c5be0cfec67ebb576d08d83f7b9/airflow/models/serialized_dag.py#L312),
 the iterator for the `else` part (which is including Postgres) is assigned as 
session.query object which is shouldn't be.
   Current :
   ```
   if session.bind.dialect.name in ["sqlite", "mysql"]:
       query = session.query(cls.dag_id, func.json_extract(cls.data, 
"$.dag.dag_dependencies"))
       iterator = ((dag_id, json.loads(deps_data) if deps_data else []) for 
dag_id, deps_data in query)
   elif session.bind.dialect.name == "mssql":
       query = session.query(cls.dag_id, func.json_query(cls.data, 
"$.dag.dag_dependencies"))
       iterator = ((dag_id, json.loads(deps_data) if deps_data else []) for 
dag_id, deps_data in query)
   else:
       iterator = session.query(cls.dag_id, func.json_extract_path(cls.data, 
"dag", "dag_dependencies"))
   return {dag_id: [DagDependency(**d) for d in (deps_data or [])] for dag_id, 
deps_data in iterator}
   ```
   
   It should be like this : 
   ```
   if session.bind.dialect.name in ["sqlite", "mysql"]:
       query = session.query(cls.dag_id, func.json_extract(cls.data, 
"$.dag.dag_dependencies"))
   elif session.bind.dialect.name == "mssql":
       query = session.query(cls.dag_id, func.json_query(cls.data, 
"$.dag.dag_dependencies"))
   else:
       query = session.query(cls.dag_id, func.json_extract_path(cls.data, 
"dag", "dag_dependencies"))
   
   iterator = ((dag_id, json.loads(deps_data) if deps_data else []) for dag_id, 
deps_data in query)
   return {dag_id: [DagDependency(**d) for d in (deps_data or [])] for dag_id, 
deps_data in iterator}
   ```
   
   ### How to reproduce
   
   1. Deploy an Airflow instance with Postgres as the meatadata DB
   2. Create two dags (one dag depends to the other), it can use 
`TriggerDagRunOperator` or  `ExternalTaskSensor`
   3. Check the Dag dependencies view from the UI : Browse ->. Dag dependencies
   
   
   ### Operating System
   
   Debian GNU/Linux 10  VERSION_ID="10" 
   
   ### Versions of Apache Airflow Providers
   
   apache-airflow                           2.2.2
   apache-airflow-providers-amazon          2.4.0
   apache-airflow-providers-celery          2.1.0
   apache-airflow-providers-cncf-kubernetes 2.1.0
   apache-airflow-providers-databricks      2.0.2
   apache-airflow-providers-docker          2.3.0
   apache-airflow-providers-elasticsearch   2.1.0
   apache-airflow-providers-ftp             2.0.1
   apache-airflow-providers-google          6.1.0
   apache-airflow-providers-grpc            2.0.1
   apache-airflow-providers-hashicorp       2.1.1
   apache-airflow-providers-http            2.0.1
   apache-airflow-providers-imap            2.0.1
   apache-airflow-providers-jira            2.0.1
   apache-airflow-providers-microsoft-azure 3.3.0
   apache-airflow-providers-microsoft-mssql 2.0.1
   apache-airflow-providers-mongo           2.2.0
   apache-airflow-providers-mysql           2.1.1
   apache-airflow-providers-odbc            2.0.1
   apache-airflow-providers-opsgenie        2.0.1
   apache-airflow-providers-postgres        2.3.0
   apache-airflow-providers-redis           2.0.1
   apache-airflow-providers-salesforce      3.3.0
   apache-airflow-providers-sendgrid        2.0.1
   apache-airflow-providers-sftp            2.2.0
   apache-airflow-providers-slack           4.1.0
   apache-airflow-providers-sqlite          2.0.1
   apache-airflow-providers-ssh             2.3.0
   apache-airflow-providers-tableau         2.1.2
   
   ### Deployment
   
   Docker-Compose
   
   ### Deployment details
   
   _No response_
   
   ### Anything else
   
   _No response_
   
   ### Are you willing to submit PR?
   
   - [X] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of 
Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to