kolfild26 opened a new issue, #24953: URL: https://github.com/apache/airflow/issues/24953
### Apache Airflow Provider(s) oracle ### Versions of Apache Airflow Providers _No response_ ### Apache Airflow version 2.3.3 (latest released) ### Operating System OEL 7.6 ### Deployment Virtualenv installation ### Deployment details _No response_ ### What happened [_map_param()](https://github.com/apache/airflow/blob/main/airflow/providers/oracle/hooks/oracle.py#L36) function from Oracle hook has an incorrect check of types: ``` PARAM_TYPES = {bool, float, int, str} def _map_param(value): if value in PARAM_TYPES: # In this branch, value is a Python type; calling it produces # an instance of the type which is understood by the Oracle driver # in the out parameter mapping mechanism. value = value() return value ``` `if value in PARAM_TYPES` never gets True for all the mentioned variables types: ``` PARAM_TYPES = {bool, float, int, str} >>> "abc" in PARAM_TYPES False >>> 123 in PARAM_TYPES False >>> True in PARAM_TYPES False >>> float(5.5) in PARAM_TYPES False ``` The correct condition would be `if type(value) in PARAM_TYPES` **But**, if we only fix this condition, next in positive case (type(value) in PARAM_TYPES = True) one more issue occurs with `value = value()` `bool`, `float`, `int` or `str` are not callable `TypeError: 'int' object is not callable` This line is probaby here for passing a python callable into sql statement of procedure params in tasks, is it? If so, need to correct: `if type(value) not in PARAM_TYPES` Here is the full fix: ``` def _map_param(value): if type(value) not in PARAM_TYPES: value = value() return value ``` Next casses are tested: ``` def oracle_callable(n=123): return n def oracle_pass(): return 123 task1 = OracleStoredProcedureOperator( task_id='task1', oracle_conn_id='oracle_conn', procedure='AIRFLOW_TEST', parameters={'var':oracle_callable} ) task2 = OracleStoredProcedureOperator( task_id='task2', oracle_conn_id='oracle_conn', procedure='AIRFLOW_TEST', parameters={'var':oracle_callable()} ) task3 = OracleStoredProcedureOperator( task_id='task3', oracle_conn_id='oracle_conn', procedure='AIRFLOW_TEST', parameters={'var':oracle_callable(456)} ) task4 = OracleStoredProcedureOperator( task_id='task4', oracle_conn_id='oracle_conn', procedure='AIRFLOW_TEST', parameters={'var':oacle_pass} ) ``` ### What you think should happen instead _No response_ ### How to reproduce _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]
