Ash Berlin-Taylor created AIRFLOW-1795: ------------------------------------------
Summary: S3Hook no longer accepts s3_conn_id breaking build in ops/sensors and back-compat Key: AIRFLOW-1795 URL: https://issues.apache.org/jira/browse/AIRFLOW-1795 Project: Apache Airflow Issue Type: Bug Affects Versions: 1.9.0 Reporter: Ash Berlin-Taylor Fix For: 1.9.0 Found whilst testing Airflow 1.9.0rc1 Previously the S3Hook accepted a parameter of {{s3_conn_id}}. As part of AIRFLOW-1520 we moved S3Hook to have a superclass of AWSHook, which accepts a {{aws_conn_id}} parameter instead. This break back-compat generally, and more specifically it breaks the built in S3KeySensor which does this: {code} def poke(self, context): from airflow.hooks.S3_hook import S3Hook hook = S3Hook(s3_conn_id=self.s3_conn_id) {code} There are a few other instances of s3_conn_id in the code base that will also probably need updating/tweaking. My first though was to add a shim mapping s3_conn_id to aws_conn_id in the S3Hook with a deprecation warning but the surface area with places where this is exposed is larger. I could add such a deprecation warning to all of these. Anyone have thoughts as to best way? - Rename all instances with deprecation warnings. - S3Hook accepts {{s3_conn_id}} and passes down to {{aws_conn_id}} in superclass. - Update existing references in code base to {{aws_conn_id}}, and not in updating about need to update in user code. (This is my least preferred option.) {noformat} airflow/operators/redshift_to_s3_operator.py 33: :param s3_conn_id: reference to a specific S3 connection 34: :type s3_conn_id: string 51: s3_conn_id='s3_default', 62: self.s3_conn_id = s3_conn_id 69: self.s3 = S3Hook(s3_conn_id=self.s3_conn_id) airflow/operators/s3_file_transform_operator.py 40: :param source_s3_conn_id: source s3 connection 41: :type source_s3_conn_id: str 44: :param dest_s3_conn_id: destination s3 connection 45: :type dest_s3_conn_id: str 62: source_s3_conn_id='s3_default', 63: dest_s3_conn_id='s3_default', 68: self.source_s3_conn_id = source_s3_conn_id 70: self.dest_s3_conn_id = dest_s3_conn_id 75: source_s3 = S3Hook(s3_conn_id=self.source_s3_conn_id) 76: dest_s3 = S3Hook(s3_conn_id=self.dest_s3_conn_id) airflow/operators/s3_to_hive_operator.py 74: :param s3_conn_id: source s3 connection 75: :type s3_conn_id: str 102: s3_conn_id='s3_default', 119: self.s3_conn_id = s3_conn_id 130: self.s3 = S3Hook(s3_conn_id=self.s3_conn_id) airflow/operators/sensors.py 504: :param s3_conn_id: a reference to the s3 connection 505: :type s3_conn_id: str 514: s3_conn_id='s3_default', 531: self.s3_conn_id = s3_conn_id 535: hook = S3Hook(s3_conn_id=self.s3_conn_id) 568: s3_conn_id='s3_default', 576: self.s3_conn_id = s3_conn_id 582: hook = S3Hook(s3_conn_id=self.s3_conn_id) {noformat} -- This message was sent by Atlassian JIRA (v6.4.14#64029)