delovermoon opened a new issue, #34794:
URL: https://github.com/apache/airflow/issues/34794
### Apache Airflow version
2.7.1
### What happened
SQLCheckOperator returns as a success no matter what as long as the query
returns a single row. This looks to be because the operator uses `all()` to
evaluate any returned values cast to false. However, the `execute` method is
checking against the full dictionary `records` (e.g. `{'DUPLICATE_ID_CHECK':
False}`) instead of just the values (e.g. `False`).
If [this
logic](https://github.com/apache/airflow/blob/59ed537fb10095176ae5bf70a19b7b48ab0850cf/airflow/providers/common/sql/operators/sql.py#L786)
was changed from:
```elif not all(records):```
to:
```elif not all(records.values()):```
that should solve the problem.
### What you think should happen instead
#### Test
Running a dummy DAG for `SQLCheckOperator` that defaults to a python `False`
boolean (e.g. `0`, `FALSE`).
#### Expectation
`SQLCheckOperator` will raise following exception -
```f"Test failed.\nQuery:\n{sql}\nResults:\n{records!s}"```
#### Actual Result
Returns `log.info("Success") and passes test
### How to reproduce
[Reproducible example from the Astronomer help
docs](https://docs.astronomer.io/learn/airflow-sql-data-quality#step-4-create-a-dag-using-sql-check-operators)
### Operating System
MacOS Ventura (13.4)
### Versions of Apache Airflow Providers
apache-airflow-providers-common-sql=1.3.2
apache-airflow-providers-snowflake=2.6.0
### Deployment
Astronomer
### Deployment details
* Using Astronomer Version 9.0.0
* Snowflake backend
* No custom configurations
### Anything else
Problem occurs every time `SQLCheckOperator` is executed.
Relevant logs:
```
[2023-10-05, 17:33:36 ] {sql.py:378} INFO - Running statement: SELECT FALSE
AS DUPLICATE_ID_CHECK
FROM DIM_TABLE;, parameters: None
[2023-10-05, 17:33:36 ] {cursor.py:804} INFO - query: [SELECT FALSE AS
DUPLICATE_ID_CHECK FROM DIM_TABLE]
[2023-10-05, 17:33:36 ] {cursor.py:817} INFO - query execution done
[2023-10-05, 17:33:36 ] {cursor.py:959} INFO - Number of results in first
chunk: 32
[2023-10-05, 17:33:36 ] {sql.py:387} INFO - Rows affected: 32
[2023-10-05, 17:33:36 ] {snowflake.py:344} INFO - Rows affected: 32
[2023-10-05, 17:33:36 ] {snowflake.py:345} INFO - Snowflake query id:
01af74a1-0403-a030-0032-4b03027d922a
[2023-10-05, 17:33:36 ] {cursor.py:804} INFO - query: [COMMIT]
[2023-10-05, 17:33:37 ] {cursor.py:817} INFO - query execution done
[2023-10-05, 17:33:37 ] {cursor.py:959} INFO - Number of results in first
chunk: 1
[2023-10-05, 17:33:37 ] {connection.py:659} INFO - closed
[2023-10-05, 17:33:38 ] {connection.py:665} INFO - No async queries seem to
be running, deleting session
***[2023-10-05, 17:33:38 ] {sql.py:776} INFO - Record:
{'DUPLICATE_ID_CHECK': False}***
***[2023-10-05, 17:33:38 ] {sql.py:782} INFO - Success.***
[2023-10-05, 17:33:39 ] {taskinstance.py:1347} INFO - Marking task as
SUCCESS. dag_id=dq_test, task_id=custom_check, execution_date=20231006T003329,
start_date=20231006T003331, end_date=20231006T003339
```
### 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]