-
Notifications
You must be signed in to change notification settings - Fork 16.4k
[AIRFLOW-3905] Allow using "parameters" in SqlSensor #4723
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||
|---|---|---|---|---|
|
|
@@ -19,6 +19,7 @@ | |||
|
|
||||
| from builtins import str | ||||
|
|
||||
| from airflow.exceptions import AirflowException | ||||
| from airflow.hooks.base_hook import BaseHook | ||||
| from airflow.sensors.base_sensor_operator import BaseSensorOperator | ||||
| from airflow.utils.decorators import apply_defaults | ||||
|
|
@@ -34,22 +35,33 @@ class SqlSensor(BaseSensorOperator): | |||
| :param sql: The sql to run. To pass, it needs to return at least one cell | ||||
| that contains a non-zero / empty string value. | ||||
| :type sql: str | ||||
| :param parameters: The parameters to render the SQL query with (optional). | ||||
| :type parameters: mapping or iterable | ||||
| """ | ||||
| template_fields = ('sql',) | ||||
| template_ext = ('.hql', '.sql',) | ||||
| ui_color = '#7c7287' | ||||
|
|
||||
| @apply_defaults | ||||
| def __init__(self, conn_id, sql, *args, **kwargs): | ||||
| self.sql = sql | ||||
| def __init__(self, conn_id, sql, parameters=None, *args, **kwargs): | ||||
| self.conn_id = conn_id | ||||
| self.sql = sql | ||||
| self.parameters = parameters | ||||
| super(SqlSensor, self).__init__(*args, **kwargs) | ||||
|
|
||||
| def poke(self, context): | ||||
| hook = BaseHook.get_connection(self.conn_id).get_hook() | ||||
| conn = BaseHook.get_connection(self.conn_id) | ||||
|
|
||||
| allowed_conn_type = {'google_cloud_platform', 'jdbc', 'mssql', | ||||
| 'mysql', 'oracle', 'postgres', | ||||
| 'presto', 'sqlite', 'vertica'} | ||||
| if conn.conn_type not in allowed_conn_type: | ||||
| raise AirflowException("The connection type is not supported by SqlSensor. " + | ||||
| "Supported connection types: {}".format(list(allowed_conn_type))) | ||||
| hook = conn.get_hook() | ||||
|
|
||||
| self.log.info('Poking: %s', self.sql) | ||||
| records = hook.get_records(self.sql) | ||||
| self.log.info('Poking: %s (with parameters %s)', self.sql, self.parameters) | ||||
| records = hook.get_records(self.sql, self.parameters) | ||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do all the sql type hooks(druid, mysql, postgres etc) are all inherited from dbapi hook? If that's case, the change LGTM
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If not, we should change that hook to inherit from dbapi hook.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, hooks of druid, mysql, mssql, Postgres, oracle etc are all inherited from dbapi hook. reference:
For
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. But I check airflow/airflow/contrib/hooks/gcp_sql_hook.py Line 669 in 6b38649
But the I assume the sqlSensor could use CloudSqlDatabasehook? Given there are many connection hooks defined, a safer approach would be check if the hook is an instance of dbapi hook, use the
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Actually Eventually
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see, but would a check on hook type be safer?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Get what you mean. Given the limitation in https://github.com/apache/airflow/blob/master/airflow/models/connection.py#L184 and the implementation of each hook, only the connection types below are supported by
I will add a check. |
||||
| if not records: | ||||
| return False | ||||
| return str(records[0][0]) not in ('0', '') | ||||
Uh oh!
There was an error while loading. Please reload this page.