From 55bcd8cddc24855fd7fcc1e509a88dcfd64dac9f Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 12 Feb 2020 13:51:03 -0500 Subject: [PATCH 001/162] Add KafkaConsumerHook, KafkaProduerHook and KafkaSensor MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Dan Ferguson dferguson992@gmail.com Co-authored-by: Hanan Shteingart Co-authored-by: YuanfΞi Zhu --- airflow/contrib/hooks/kafka_consumer_hook.py | 69 ++++++++++++++++++++ airflow/contrib/hooks/kafka_producer_hook.py | 56 ++++++++++++++++ airflow/contrib/sensors/kafka_sensor.py | 50 ++++++++++++++ 3 files changed, 175 insertions(+) create mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py create mode 100644 airflow/contrib/hooks/kafka_producer_hook.py create mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py new file mode 100644 index 0000000000000..335c267b7837f --- /dev/null +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -0,0 +1,69 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaConsumer + + +class KafkaConsumerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + # Disable auto commit as the hook will commit right + # after polling. + conf['enable_auto_commit'] = False + + self.server = '{host}:{port}'.format(**locals()) + self.consumer = KafkaConsumer( + self.topic, + bootstrap_servers=self.server, **conf) + + return self.consumer + + def get_messages(self): + """ + Get all the messages haven't been consumed, it doesn't + block by default, then commit the offset. + :return: + A list of messages + """ + consumer = self.get_conn() + + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms=50) + + consumer.commit() + return messages + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.consumer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py new file mode 100644 index 0000000000000..ffa5d4693e9c2 --- /dev/null +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -0,0 +1,56 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaProducer + + +class KafkaProducerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + conf['enable_auto_commit'] = False + self.server = '{host}:{port}'.format(**locals()) + self.producer = KafkaProducer( + bootstrap_servers=self.server, **conf) + + return self.producer + + def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): + producer = self.get_pub() + future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, + timestamp_ms=timestamp_ms) + producer.close() + return future_record_metadata + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.produer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py new file mode 100644 index 0000000000000..738e63c76c730 --- /dev/null +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -0,0 +1,50 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import logging + +from src.kafka_hook import KafkaConsumerHook +from airflow.operators.sensors import BaseSensorOperator +from airflow.utils import apply_defaults + + +class KafkaSensor(BaseSensorOperator): + """ + Consumes the Kafka message with the specific topic + """ + + @apply_defaults + def __init__(self, conn_id, topic, *args, **kwargs): + """ + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. + :param topic: + the subscribed topic + """ + self.topic = topic + self.hook = KafkaConsumerHook(conn_id, topic) + super(KafkaSensor, self).__init__(*args, **kwargs) + + def poke(self, context): + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) + + messages = self.hook.get_messages() + + logging.info( + 'Got messages during poking: %s', messages) + + return messages or False From 3706a598a1a903ad902f815e4f5ec5b3ee862d2c Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 18 Feb 2020 09:05:42 -0500 Subject: [PATCH 002/162] Modifying files in accordance with peer review --- airflow/contrib/hooks/kafka_consumer_hook.py | 24 +++++++++++--------- airflow/contrib/hooks/kafka_producer_hook.py | 24 +++++++++++--------- airflow/contrib/sensors/kafka_sensor.py | 16 +++++++++---- 3 files changed, 38 insertions(+), 26 deletions(-) diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py index 335c267b7837f..1143e2ccfb6a7 100644 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -16,8 +16,8 @@ class KafkaConsumerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): super(KafkaConsumerHook, self).__init__(None) @@ -29,21 +29,21 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT # Disable auto commit as the hook will commit right # after polling. conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.consumer = KafkaConsumer( self.topic, bootstrap_servers=self.server, **conf) return self.consumer - def get_messages(self): + def get_messages(self, timeout_ms=50): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. @@ -51,12 +51,14 @@ def get_messages(self): A list of messages """ consumer = self.get_conn() + try: + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms) - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms=50) - - consumer.commit() + consumer.commit() + finally: + consumer.close() return messages def __repr__(self): diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py index ffa5d4693e9c2..9c1ebb158afe5 100644 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -16,11 +16,11 @@ class KafkaProducerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) + super(KafkaProducerHook, self).__init__(None) self.conn = self.get_connection(conn_id) self.server = None self.consumer = None @@ -29,21 +29,23 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.producer = KafkaProducer( bootstrap_servers=self.server, **conf) return self.producer - def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): - producer = self.get_pub() - future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, - timestamp_ms=timestamp_ms) - producer.close() + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + producer = self.get_conn() + try: + future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, + timestamp_ms=timestamp_ms) + finally: + producer.close() return future_record_metadata def __repr__(self): diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py index 738e63c76c730..e51328cdd63a9 100644 --- a/airflow/contrib/sensors/kafka_sensor.py +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -12,9 +12,11 @@ import logging -from src.kafka_hook import KafkaConsumerHook -from airflow.operators.sensors import BaseSensorOperator -from airflow.utils import apply_defaults +from werkzeug.utils import cached_property + +from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class KafkaSensor(BaseSensorOperator): @@ -22,6 +24,8 @@ class KafkaSensor(BaseSensorOperator): Consumes the Kafka message with the specific topic """ + templated_fields = ('topic',) + @apply_defaults def __init__(self, conn_id, topic, *args, **kwargs): """ @@ -34,9 +38,13 @@ def __init__(self, conn_id, topic, *args, **kwargs): the subscribed topic """ self.topic = topic - self.hook = KafkaConsumerHook(conn_id, topic) + self.conn_id = conn_id super(KafkaSensor, self).__init__(*args, **kwargs) + @cached_property + def hook(self): + return KafkaConsumerHook(self.conn_id, self.topic) + def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', From 3ed7b7790d54f53a3f973f2b5f61ae3e52e622a0 Mon Sep 17 00:00:00 2001 From: dan Date: Mon, 2 Mar 2020 15:47:10 -0500 Subject: [PATCH 003/162] Moved files to providers package. Externalized connection logic from init function --- airflow/providers/apache/kafka/__init__.py | 11 +++++ .../apache/kafka/example_dags/__init__.py | 11 +++++ .../kafka/example_dags/example_kafka_dag.py | 28 +++++++++++++ .../providers/apache/kafka/hooks/__init__.py | 11 +++++ .../kafka}/hooks/kafka_consumer_hook.py | 41 +++++++++---------- .../kafka}/hooks/kafka_producer_hook.py | 22 +++++----- .../apache/kafka/sensors/__init__.py | 11 +++++ .../apache/kafka}/sensors/kafka_sensor.py | 32 +++++++++------ 8 files changed, 123 insertions(+), 44 deletions(-) create mode 100644 airflow/providers/apache/kafka/__init__.py create mode 100644 airflow/providers/apache/kafka/example_dags/__init__.py create mode 100644 airflow/providers/apache/kafka/example_dags/example_kafka_dag.py create mode 100644 airflow/providers/apache/kafka/hooks/__init__.py rename airflow/{contrib => providers/apache/kafka}/hooks/kafka_consumer_hook.py (65%) rename airflow/{contrib => providers/apache/kafka}/hooks/kafka_producer_hook.py (79%) create mode 100644 airflow/providers/apache/kafka/sensors/__init__.py rename airflow/{contrib => providers/apache/kafka}/sensors/kafka_sensor.py (67%) diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py new file mode 100644 index 0000000000000..164f68b09175b --- /dev/null +++ b/airflow/providers/apache/kafka/__init__.py @@ -0,0 +1,11 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/airflow/providers/apache/kafka/example_dags/__init__.py b/airflow/providers/apache/kafka/example_dags/__init__.py new file mode 100644 index 0000000000000..164f68b09175b --- /dev/null +++ b/airflow/providers/apache/kafka/example_dags/__init__.py @@ -0,0 +1,11 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py new file mode 100644 index 0000000000000..6b053d007ac56 --- /dev/null +++ b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py @@ -0,0 +1,28 @@ + +from airflow import DAG +from datetime import timedelta + +from airflow.utils.timezone import datetime +from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor + +DAG_ID = "example_kafka_dag" +dag_start_date = datetime(2015, 6, 1, hour=20, tzinfo=None) +default_args = { + 'owner': '@Ferg_In', + 'depends_on_past': False, + 'start_date': dag_start_date, + 'email': ['dferguson992@gmail.com'], + 'provide_context': True, + 'retries': 3, + 'retry_delay': timedelta(minutes=5), +} + +with DAG(dag_id=DAG_ID, default_args=default_args, schedule_interval=None, + max_active_runs=1, concurrency=4, catchup=False) as dag: + + sensor = KafkaSensor( + task_id='trigger', + topic='', + host='', + port='' + ) diff --git a/airflow/providers/apache/kafka/hooks/__init__.py b/airflow/providers/apache/kafka/hooks/__init__.py new file mode 100644 index 0000000000000..164f68b09175b --- /dev/null +++ b/airflow/providers/apache/kafka/hooks/__init__.py @@ -0,0 +1,11 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py similarity index 65% rename from airflow/contrib/hooks/kafka_consumer_hook.py rename to airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 1143e2ccfb6a7..cbb2a4c963839 100644 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -16,34 +16,36 @@ class KafkaConsumerHook(BaseHook): - DEFAULT_HOST = 'localhost' + DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 - def __init__(self, conn_id, topic): + def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='kafka_default'): super(KafkaConsumerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) + self.conn_id = kafka_conn_id + self._conn = None self.server = None self.consumer = None - self.producer = None + self.extra_dejson = {} self.topic = topic + self.host = host + self.port = port def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - # Disable auto commit as the hook will commit right - # after polling. - conf['enable_auto_commit'] = False - - self.server = f"""{host}:{port}""" - self.consumer = KafkaConsumer( - self.topic, - bootstrap_servers=self.server, **conf) + if not self._conn: + conn = self.get_connection(self.conn_id) + service_options = conn.extra_dejson + host = conn.host or self.DEFAULT_HOST + port = conn.port or self.DEFAULT_PORT + self.server = f"""{host}:{port}""" + self.consumer = KafkaConsumer( + self.topic, + bootstrap_servers=self.server, + **service_options + ) return self.consumer - def get_messages(self, timeout_ms=50): + def get_messages(self, timeout_ms=5000): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. @@ -52,11 +54,8 @@ def get_messages(self, timeout_ms=50): """ consumer = self.get_conn() try: - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. messages = consumer.poll(timeout_ms) - - consumer.commit() + # consumer.commit() finally: consumer.close() return messages diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py similarity index 79% rename from airflow/contrib/hooks/kafka_producer_hook.py rename to airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 9c1ebb158afe5..4bc60e2a610fd 100644 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -21,22 +21,24 @@ class KafkaProducerHook(BaseHook): def __init__(self, conn_id, topic): super(KafkaProducerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) + self.conn = None self.server = None self.consumer = None self.producer = None self.topic = topic def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - conf['enable_auto_commit'] = False - self.server = f"""{host}:{port}""" - self.producer = KafkaProducer( - bootstrap_servers=self.server, **conf) - + if not self._conn: + conn = self.get_connection(self.conn_id) + service_options = conn.extra_dejson + host = conn.host or self.DEFAULT_HOST + port = conn.port or self.DEFAULT_PORT + + self.server = f"""{host}:{port}""" + self.consumer = KafkaProducer( + bootstrap_servers=self.server, + **service_options + ) return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): diff --git a/airflow/providers/apache/kafka/sensors/__init__.py b/airflow/providers/apache/kafka/sensors/__init__.py new file mode 100644 index 0000000000000..164f68b09175b --- /dev/null +++ b/airflow/providers/apache/kafka/sensors/__init__.py @@ -0,0 +1,11 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py similarity index 67% rename from airflow/contrib/sensors/kafka_sensor.py rename to airflow/providers/apache/kafka/sensors/kafka_sensor.py index e51328cdd63a9..f7043a0294706 100644 --- a/airflow/contrib/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -12,22 +12,25 @@ import logging -from werkzeug.utils import cached_property - -from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook -from airflow.sensors.base_sensor_operator import BaseSensorOperator +from cached_property import cached_property from airflow.utils.decorators import apply_defaults +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook class KafkaSensor(BaseSensorOperator): """ Consumes the Kafka message with the specific topic """ - - templated_fields = ('topic',) + DEFAULT_HOST = 'kafka1' + DEFAULT_PORT = 9092 + templated_fields = ('topic', + 'host', + 'port', + ) @apply_defaults - def __init__(self, conn_id, topic, *args, **kwargs): + def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs): """ Initialize the sensor, the connection establish is put off to it's first time usage. @@ -38,12 +41,13 @@ def __init__(self, conn_id, topic, *args, **kwargs): the subscribed topic """ self.topic = topic - self.conn_id = conn_id + self.host = host + self.port = port super(KafkaSensor, self).__init__(*args, **kwargs) @cached_property def hook(self): - return KafkaConsumerHook(self.conn_id, self.topic) + return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): logging.info( @@ -52,7 +56,9 @@ def poke(self, context): messages = self.hook.get_messages() - logging.info( - 'Got messages during poking: %s', messages) - - return messages or False + if messages is not {}: + logging.info( + 'Got messages during poking: %s', messages) + return messages + else: + return False From 7c50651f58e62778d036a6a0ab3382d43fed1df4 Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 11 Mar 2020 09:06:55 -0400 Subject: [PATCH 004/162] Fixing Travis CI Linting errors --- airflow/providers/apache/kafka/__init__.py | 24 ++++++++++++------- .../providers/apache/kafka/hooks/__init__.py | 24 ++++++++++++------- .../apache/kafka/hooks/kafka_consumer_hook.py | 2 +- .../apache/kafka/hooks/kafka_producer_hook.py | 2 +- .../apache/kafka/sensors/kafka_sensor.py | 24 ++++++++++++------- 5 files changed, 47 insertions(+), 29 deletions(-) diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py index 164f68b09175b..217e5db960782 100644 --- a/airflow/providers/apache/kafka/__init__.py +++ b/airflow/providers/apache/kafka/__init__.py @@ -1,11 +1,17 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/airflow/providers/apache/kafka/hooks/__init__.py b/airflow/providers/apache/kafka/hooks/__init__.py index 164f68b09175b..217e5db960782 100644 --- a/airflow/providers/apache/kafka/hooks/__init__.py +++ b/airflow/providers/apache/kafka/hooks/__init__.py @@ -1,11 +1,17 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index cbb2a4c963839..968514398a20b 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -10,8 +10,8 @@ # See the License for the specific language governing permissions and # limitations under the License. -from airflow.hooks import base_hook as BaseHook from kafka import KafkaConsumer +from airflow.hooks.base_hook import BaseHook class KafkaConsumerHook(BaseHook): diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 4bc60e2a610fd..7d74110012529 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -10,8 +10,8 @@ # See the License for the specific language governing permissions and # limitations under the License. -from airflow.hooks import base_hook as BaseHook from kafka import KafkaProducer +from airflow.hooks.base_hook import BaseHook class KafkaProducerHook(BaseHook): diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index f7043a0294706..c5096d8734fcd 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -1,14 +1,20 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. import logging From 89f576458f0ef7cb9003a24b05b8e654c4370495 Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 11 Mar 2020 09:06:55 -0400 Subject: [PATCH 005/162] Fixing Travis CI Linting errors --- .../apache/kafka/example_dags/__init__.py | 24 ++++++++----- .../kafka/example_dags/example_kafka_dag.py | 17 +++++++++ .../apache/kafka/hooks/kafka_consumer_hook.py | 25 ++++++++----- .../apache/kafka/hooks/kafka_producer_hook.py | 35 +++++++++++-------- .../apache/kafka/sensors/__init__.py | 24 ++++++++----- 5 files changed, 84 insertions(+), 41 deletions(-) diff --git a/airflow/providers/apache/kafka/example_dags/__init__.py b/airflow/providers/apache/kafka/example_dags/__init__.py index 164f68b09175b..217e5db960782 100644 --- a/airflow/providers/apache/kafka/example_dags/__init__.py +++ b/airflow/providers/apache/kafka/example_dags/__init__.py @@ -1,11 +1,17 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py index 6b053d007ac56..eb89df6b7cca8 100644 --- a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py +++ b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. from airflow import DAG from datetime import timedelta diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 968514398a20b..bb4895fab25fb 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -1,14 +1,21 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + from kafka import KafkaConsumer from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 7d74110012529..e7f70f2ef1aec 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -1,14 +1,20 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. from kafka import KafkaProducer from airflow.hooks.base_hook import BaseHook @@ -21,7 +27,8 @@ class KafkaProducerHook(BaseHook): def __init__(self, conn_id, topic): super(KafkaProducerHook, self).__init__(None) - self.conn = None + self.conn_id = conn_id + self._conn = None self.server = None self.consumer = None self.producer = None @@ -29,10 +36,10 @@ def __init__(self, conn_id, topic): def get_conn(self): if not self._conn: - conn = self.get_connection(self.conn_id) - service_options = conn.extra_dejson - host = conn.host or self.DEFAULT_HOST - port = conn.port or self.DEFAULT_PORT + _conn = self.get_connection(self.conn_id) + service_options = _conn.extra_dejson + host = _conn.host or self.DEFAULT_HOST + port = _conn.port or self.DEFAULT_PORT self.server = f"""{host}:{port}""" self.consumer = KafkaProducer( diff --git a/airflow/providers/apache/kafka/sensors/__init__.py b/airflow/providers/apache/kafka/sensors/__init__.py index 164f68b09175b..217e5db960782 100644 --- a/airflow/providers/apache/kafka/sensors/__init__.py +++ b/airflow/providers/apache/kafka/sensors/__init__.py @@ -1,11 +1,17 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. From 0874dc25ea70139fd8678b965169712b1c9e6dda Mon Sep 17 00:00:00 2001 From: dan Date: Thu, 30 Apr 2020 09:57:14 -0400 Subject: [PATCH 006/162] Fixed isort issue --- .../providers/apache/kafka/example_dags/example_kafka_dag.py | 4 ++-- airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py | 1 + airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 1 + airflow/providers/apache/kafka/sensors/kafka_sensor.py | 5 +++-- 4 files changed, 7 insertions(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py index eb89df6b7cca8..da333206b6423 100644 --- a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py +++ b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py @@ -16,11 +16,11 @@ # specific language governing permissions and limitations # under the License. -from airflow import DAG from datetime import timedelta -from airflow.utils.timezone import datetime +from airflow import DAG from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor +from airflow.utils.timezone import datetime DAG_ID = "example_kafka_dag" dag_start_date = datetime(2015, 6, 1, hour=20, tzinfo=None) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index bb4895fab25fb..c5208508a6e3d 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -18,6 +18,7 @@ from kafka import KafkaConsumer + from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index e7f70f2ef1aec..f7410ee097fb7 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -17,6 +17,7 @@ # under the License. from kafka import KafkaProducer + from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index c5096d8734fcd..8649f876407ec 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -19,9 +19,10 @@ import logging from cached_property import cached_property -from airflow.utils.decorators import apply_defaults -from airflow.sensors.base_sensor_operator import BaseSensorOperator + from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class KafkaSensor(BaseSensorOperator): From 531a56b746e2fba54c8a379f25b0b6ed0f38770e Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:16:08 -0400 Subject: [PATCH 007/162] produer --> producer --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index f7410ee097fb7..5272de1cbf1bc 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -62,7 +62,7 @@ def __repr__(self): """ Pretty the hook with the connection info """ - connected = self.produer is not None + connected = self.producer is not None return '' % \ (connected, self.server, self.topic) From 0921de040f1e7b5c0f9aebe6e5ef2f518748bf30 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:36:16 -0400 Subject: [PATCH 008/162] Added pylint mandated doc strings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 2 ++ .../apache/kafka/hooks/kafka_producer_hook.py | 11 +++++++++++ .../providers/apache/kafka/sensors/kafka_sensor.py | 7 ++++++- 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index c5208508a6e3d..d7cd9b2d175cb 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -23,7 +23,9 @@ class KafkaConsumerHook(BaseHook): + """ + """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 5272de1cbf1bc..35232c253b819 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -22,7 +22,9 @@ class KafkaProducerHook(BaseHook): + """ + """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 @@ -50,6 +52,15 @@ def get_conn(self): return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + """ + + :param topic: + :param value: + :param key: + :param partition: + :param timestamp_ms: + :return: + """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 8649f876407ec..e4c33f5fa16f8 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -54,12 +54,17 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): + """ + + :return: + KafkaConsumerHook + """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', - self.topic, self.hook) + str(self.topic), str(self.hook)) messages = self.hook.get_messages() From 37d5d1f6859961942d3a5fd827a24445d71ea872 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 10:06:05 -0400 Subject: [PATCH 009/162] Fixing more pylint issues --- airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py | 2 +- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index d7cd9b2d175cb..025ebc1077668 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -24,7 +24,7 @@ class KafkaConsumerHook(BaseHook): """ - + KafkaConsumerHook Class. """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 35232c253b819..affa37ef3ac05 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -23,7 +23,7 @@ class KafkaProducerHook(BaseHook): """ - + KafkaProducerHook Class. """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index e4c33f5fa16f8..250656398ac4e 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -68,9 +68,9 @@ def poke(self, context): messages = self.hook.get_messages() - if messages is not {}: + if not messages: logging.info( - 'Got messages during poking: %s', messages) + 'Got messages during poking: %s', str(messages)) return messages else: return False From 26227a700757d819ff931d535a3bd1604bad2d6f Mon Sep 17 00:00:00 2001 From: dan Date: Sat, 23 May 2020 14:53:06 -0400 Subject: [PATCH 010/162] Removed unused imports --- docs/operators-and-hooks-ref.rst | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/operators-and-hooks-ref.rst b/docs/operators-and-hooks-ref.rst index 8e71c6e37e84b..fa29aad6a91bf 100644 --- a/docs/operators-and-hooks-ref.rst +++ b/docs/operators-and-hooks-ref.rst @@ -142,6 +142,12 @@ Foundation. :mod:`airflow.providers.apache.druid.operators.druid_check` - + * - `Apache Kafka `__ + - + - :mod:`airflow.providers.apache.kafka.hooks.kafka_producer_hook` + - :mod:`airflow.providers.apache.kafka.hooks.kafka_consumer_hook` + - :mod:`airflow.providers.apache.kafka.sensors.kafka_sensor` + * - `Apache Hive `__ - - :mod:`airflow.providers.apache.hive.hooks.hive` From d2a4b220913f44b5423d18be53436bae8c0064a8 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 11:57:34 -0400 Subject: [PATCH 011/162] Added comma in documentation --- docs/operators-and-hooks-ref.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/operators-and-hooks-ref.rst b/docs/operators-and-hooks-ref.rst index fa29aad6a91bf..1b5128b65b8af 100644 --- a/docs/operators-and-hooks-ref.rst +++ b/docs/operators-and-hooks-ref.rst @@ -144,7 +144,7 @@ Foundation. * - `Apache Kafka `__ - - - :mod:`airflow.providers.apache.kafka.hooks.kafka_producer_hook` + - :mod:`airflow.providers.apache.kafka.hooks.kafka_producer_hook`, - :mod:`airflow.providers.apache.kafka.hooks.kafka_consumer_hook` - :mod:`airflow.providers.apache.kafka.sensors.kafka_sensor` From ecb8a31f299f54090528754b0e0d5e9ca1626ea0 Mon Sep 17 00:00:00 2001 From: dan Date: Thu, 7 May 2020 10:33:57 -0400 Subject: [PATCH 012/162] Modified the templates index file to see the new hooks and sensors --- docs/autoapi_templates/index.rst | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/autoapi_templates/index.rst b/docs/autoapi_templates/index.rst index a1263861c25e5..cb4eb74c116a5 100644 --- a/docs/autoapi_templates/index.rst +++ b/docs/autoapi_templates/index.rst @@ -88,6 +88,10 @@ All operators are in the following packages: airflow/providers/apache/kylin/operators/index + airflow/providers/apache/kafka/sensors/index + + airflow/providers/apache/kafka/hooks/index + airflow/providers/apache/livy/operators/index airflow/providers/apache/livy/sensors/index From f3b1086ec33bb4192227f8d68bae3f21136a83fc Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 013/162] Tried to fix docstrings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 17 +++++++++++--- .../apache/kafka/hooks/kafka_producer_hook.py | 17 +++++++++++--- .../apache/kafka/sensors/kafka_sensor.py | 22 +++++++++++++------ 3 files changed, 43 insertions(+), 13 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 025ebc1077668..df2576069e927 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -40,7 +40,13 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self): + def get_conn(self) -> KafkaConsumer: + """ + A Kafka Consumer object. + + :return: + A Kafka Consumer object. + """ if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -55,10 +61,12 @@ def get_conn(self): ) return self.consumer - def get_messages(self, timeout_ms=5000): + def get_messages(self, timeout_ms=5000) -> dict: """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. + + :param timeout_ms: :return: A list of messages """ @@ -72,7 +80,10 @@ def get_messages(self, timeout_ms=5000): def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.consumer is not None return ' KafkaProducer: + """ + Returns a Kafka Producer + + :return: + A Kafka Producer object. + """ if not self._conn: _conn = self.get_connection(self.conn_id) service_options = _conn.extra_dejson @@ -51,8 +58,9 @@ def get_conn(self): ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: """ + Sends a message on the specified topic and partition. Keyed messages will be sent in order. :param topic: :param value: @@ -71,7 +79,10 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.producer is not None return ' Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 014/162] Tried to fix docstrings --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 6bef4e4b56381..9919505b4a24d 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -58,7 +58,7 @@ def get_conn(self) -> KafkaProducer: ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): """ Sends a message on the specified topic and partition. Keyed messages will be sent in order. From 9d379f0be7bd0aaf071073a60a2d1fac22af23b9 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 9 May 2020 23:41:58 +0100 Subject: [PATCH 015/162] Apply suggestions from code review --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 50ca55079d9ac..60b95989b64cb 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -56,10 +56,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): """ - Returns a Kafka Consumer Hook - - :return: - KafkaConsumerHook + Returns a Kafka Consumer Hook """ return KafkaConsumerHook(self.topic, self.host, self.port) From 206f9d995fe02a02faf313aae595877856bd07fb Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:28 -0400 Subject: [PATCH 016/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 60b95989b64cb..7108f84207bcd 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -74,8 +74,7 @@ def poke(self, context): messages = self.hook.get_messages() if messages: - logging.info( - 'Got messages during poking: %s', str(messages)) + self.log.info('Got messages during poking: %s', str(messages)) return messages else: return False From 5885b1401bdd1a05f57bab657b4f8db650083aea Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:46 -0400 Subject: [PATCH 017/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 7108f84207bcd..bb27e2f5adc73 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -67,9 +67,7 @@ def poke(self, context): :param context: :return: """ - logging.info( - 'Poking topic: %s, using hook: %s', - str(self.topic), str(self.hook)) + self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) messages = self.hook.get_messages() From 192a93d2ccf30285703e2db4d5d7f0cba5d3c51e Mon Sep 17 00:00:00 2001 From: dan Date: Sat, 23 May 2020 14:46:59 -0400 Subject: [PATCH 018/162] Removed unused imports --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 1 - airflow/providers/apache/kafka/sensors/kafka_sensor.py | 2 -- 2 files changed, 3 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 9919505b4a24d..f6d9fd2428ec0 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -17,7 +17,6 @@ # under the License. from kafka import KafkaProducer -from kafka.producer.future import FutureRecordMetadata from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index bb27e2f5adc73..842643c60ba40 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -16,8 +16,6 @@ # specific language governing permissions and limitations # under the License. -import logging - from cached_property import cached_property from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook From c221956fdf6ad79eb978e3c9d4783477de3456bd Mon Sep 17 00:00:00 2001 From: dan Date: Sat, 23 May 2020 14:46:59 -0400 Subject: [PATCH 019/162] Removed unused imports --- CONTRIBUTING.rst | 4 ++-- setup.py | 5 +++++ 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/CONTRIBUTING.rst b/CONTRIBUTING.rst index cb5d600dfbeaa..3fc8100d21832 100644 --- a/CONTRIBUTING.rst +++ b/CONTRIBUTING.rst @@ -432,8 +432,8 @@ This is the full list of those extras: .. START EXTRAS HERE -all_dbs, amazon, apache.atlas, apache.beam, apache.cassandra, apache.druid, apache.hdfs, -apache.hive, apache.kylin, apache.pinot, apache.webhdfs, async, atlas, aws, azure, cassandra, +all_dbs, amazon, apache.atlas, apache_beam, apache.cassandra, apache.druid, apache.hdfs, +apache.hive, apache.kafka, apache.pinot, apache.webhdfs, async, atlas, aws, azure, cassandra, celery, cgroups, cloudant, cncf.kubernetes, dask, databricks, datadog, devel, devel_hadoop, doc, docker, druid, elasticsearch, exasol, facebook, gcp, gcp_api, github_enterprise, google, google_auth, grpc, hashicorp, hdfs, hive, jdbc, jira, kerberos, kubernetes, ldap, microsoft.azure, diff --git a/setup.py b/setup.py index 44f72c12c9bb2..a294c7a0a7ff4 100644 --- a/setup.py +++ b/setup.py @@ -312,6 +312,9 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version jira = [ 'JIRA>1.0.7', ] +kafka = [ + 'kafka>=1.3.5' +] kerberos = [ 'pykerberos>=1.1.13', 'requests_kerberos>=0.10.0', @@ -498,6 +501,7 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version "apache.hdfs": hdfs, "apache.hive": hive, "apache.kylin": kylin, + "apache.kafka": kafka, "apache.livy": [], "apache.pig": [], "apache.pinot": pinot, @@ -562,6 +566,7 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version "apache.hdfs": hdfs, "apache.hive": hive, "apache.kylin": kylin, + "apache.kafka": kafka, "apache.pinot": pinot, "apache.webhdfs": webhdfs, 'async': async_packages, From 7fe057ef45c85748f79f6cba42a0c09e378cb11f Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 10:54:44 -0400 Subject: [PATCH 020/162] Added new line to readme to trigger builds --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index c2869acb4661f..647b9f9ac4599 100644 --- a/README.md +++ b/README.md @@ -16,6 +16,7 @@ specific language governing permissions and limitations under the License. --> + # Apache Airflow [![PyPI version](https://badge.fury.io/py/apache-airflow.svg)](https://badge.fury.io/py/apache-airflow) From 788892dbc0df2f01054be76c381d5382c838d2dd Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 16:12:11 -0400 Subject: [PATCH 021/162] added the python 3.8 requirements --- requirements/requirements-python3.6.txt | 5 +++-- requirements/requirements-python3.7.txt | 5 +++-- requirements/requirements-python3.8.txt | 1 + 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index 91051f5330482..562c98c295111 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -56,7 +56,7 @@ azure-batch==9.0.0 azure-common==1.1.25 azure-cosmos==3.2.0 azure-datalake-store==0.0.48 -azure-kusto-data==0.0.45 +azure-kusto-data==0.1.0 azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 @@ -163,7 +163,7 @@ google-cloud-spanner==1.17.1 google-cloud-speech==1.3.2 google-cloud-storage==1.29.0 google-cloud-tasks==1.5.0 -google-cloud-texttospeech==1.0.1 +google-cloud-texttospeech==2.0.0 google-cloud-translate==2.0.1 google-cloud-videointelligence==1.15.0 google-cloud-vision==1.0.0 @@ -207,6 +207,7 @@ jsonschema==3.2.0 junit-xml==1.9 jupyter-client==6.1.6 jupyter-core==4.6.3 +kafka==1.3.5 kombu==4.6.11 kubernetes==11.0.0 kylinpy==2.8.1 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index 5bbc3b08cce43..8d6e7542a4821 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -56,7 +56,7 @@ azure-batch==9.0.0 azure-common==1.1.25 azure-cosmos==3.2.0 azure-datalake-store==0.0.48 -azure-kusto-data==0.0.45 +azure-kusto-data==0.1.0 azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 @@ -162,7 +162,7 @@ google-cloud-spanner==1.17.1 google-cloud-speech==1.3.2 google-cloud-storage==1.29.0 google-cloud-tasks==1.5.0 -google-cloud-texttospeech==1.0.1 +google-cloud-texttospeech==2.0.0 google-cloud-translate==2.0.1 google-cloud-videointelligence==1.15.0 google-cloud-vision==1.0.0 @@ -203,6 +203,7 @@ jsonschema==3.2.0 junit-xml==1.9 jupyter-client==6.1.6 jupyter-core==4.6.3 +kafka==1.3.5 kombu==4.6.11 kubernetes==11.0.0 kylinpy==2.8.1 diff --git a/requirements/requirements-python3.8.txt b/requirements/requirements-python3.8.txt index 864fbcd0c0134..ccde845c95c99 100644 --- a/requirements/requirements-python3.8.txt +++ b/requirements/requirements-python3.8.txt @@ -203,6 +203,7 @@ jsonschema==3.2.0 junit-xml==1.9 jupyter-client==6.1.6 jupyter-core==4.6.3 +kafka==1.3.5 kombu==4.6.11 kubernetes==11.0.0 kylinpy==2.8.1 From e792bbb0ad574f987013a8eb8be604cc0bd7b84a Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 12 Feb 2020 13:51:03 -0500 Subject: [PATCH 022/162] Add KafkaConsumerHook, KafkaProduerHook and KafkaSensor MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Dan Ferguson dferguson992@gmail.com Co-authored-by: Hanan Shteingart Co-authored-by: YuanfΞi Zhu --- airflow/contrib/hooks/kafka_consumer_hook.py | 69 ++++++++++++++++++++ airflow/contrib/hooks/kafka_producer_hook.py | 56 ++++++++++++++++ airflow/contrib/sensors/kafka_sensor.py | 50 ++++++++++++++ 3 files changed, 175 insertions(+) create mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py create mode 100644 airflow/contrib/hooks/kafka_producer_hook.py create mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py new file mode 100644 index 0000000000000..335c267b7837f --- /dev/null +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -0,0 +1,69 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaConsumer + + +class KafkaConsumerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + # Disable auto commit as the hook will commit right + # after polling. + conf['enable_auto_commit'] = False + + self.server = '{host}:{port}'.format(**locals()) + self.consumer = KafkaConsumer( + self.topic, + bootstrap_servers=self.server, **conf) + + return self.consumer + + def get_messages(self): + """ + Get all the messages haven't been consumed, it doesn't + block by default, then commit the offset. + :return: + A list of messages + """ + consumer = self.get_conn() + + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms=50) + + consumer.commit() + return messages + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.consumer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py new file mode 100644 index 0000000000000..ffa5d4693e9c2 --- /dev/null +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -0,0 +1,56 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaProducer + + +class KafkaProducerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + conf['enable_auto_commit'] = False + self.server = '{host}:{port}'.format(**locals()) + self.producer = KafkaProducer( + bootstrap_servers=self.server, **conf) + + return self.producer + + def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): + producer = self.get_pub() + future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, + timestamp_ms=timestamp_ms) + producer.close() + return future_record_metadata + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.produer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py new file mode 100644 index 0000000000000..738e63c76c730 --- /dev/null +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -0,0 +1,50 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import logging + +from src.kafka_hook import KafkaConsumerHook +from airflow.operators.sensors import BaseSensorOperator +from airflow.utils import apply_defaults + + +class KafkaSensor(BaseSensorOperator): + """ + Consumes the Kafka message with the specific topic + """ + + @apply_defaults + def __init__(self, conn_id, topic, *args, **kwargs): + """ + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. + :param topic: + the subscribed topic + """ + self.topic = topic + self.hook = KafkaConsumerHook(conn_id, topic) + super(KafkaSensor, self).__init__(*args, **kwargs) + + def poke(self, context): + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) + + messages = self.hook.get_messages() + + logging.info( + 'Got messages during poking: %s', messages) + + return messages or False From 8cf49a1bb1362a63b6bc91f61dda69099bcfcab9 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 18 Feb 2020 09:05:42 -0500 Subject: [PATCH 023/162] Modifying files in accordance with peer review --- airflow/contrib/hooks/kafka_consumer_hook.py | 24 +++++++++++--------- airflow/contrib/hooks/kafka_producer_hook.py | 24 +++++++++++--------- airflow/contrib/sensors/kafka_sensor.py | 16 +++++++++---- 3 files changed, 38 insertions(+), 26 deletions(-) diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py index 335c267b7837f..1143e2ccfb6a7 100644 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -16,8 +16,8 @@ class KafkaConsumerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): super(KafkaConsumerHook, self).__init__(None) @@ -29,21 +29,21 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT # Disable auto commit as the hook will commit right # after polling. conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.consumer = KafkaConsumer( self.topic, bootstrap_servers=self.server, **conf) return self.consumer - def get_messages(self): + def get_messages(self, timeout_ms=50): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. @@ -51,12 +51,14 @@ def get_messages(self): A list of messages """ consumer = self.get_conn() + try: + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms) - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms=50) - - consumer.commit() + consumer.commit() + finally: + consumer.close() return messages def __repr__(self): diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py index ffa5d4693e9c2..9c1ebb158afe5 100644 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -16,11 +16,11 @@ class KafkaProducerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) + super(KafkaProducerHook, self).__init__(None) self.conn = self.get_connection(conn_id) self.server = None self.consumer = None @@ -29,21 +29,23 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.producer = KafkaProducer( bootstrap_servers=self.server, **conf) return self.producer - def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): - producer = self.get_pub() - future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, - timestamp_ms=timestamp_ms) - producer.close() + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + producer = self.get_conn() + try: + future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, + timestamp_ms=timestamp_ms) + finally: + producer.close() return future_record_metadata def __repr__(self): diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py index 738e63c76c730..e51328cdd63a9 100644 --- a/airflow/contrib/sensors/kafka_sensor.py +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -12,9 +12,11 @@ import logging -from src.kafka_hook import KafkaConsumerHook -from airflow.operators.sensors import BaseSensorOperator -from airflow.utils import apply_defaults +from werkzeug.utils import cached_property + +from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class KafkaSensor(BaseSensorOperator): @@ -22,6 +24,8 @@ class KafkaSensor(BaseSensorOperator): Consumes the Kafka message with the specific topic """ + templated_fields = ('topic',) + @apply_defaults def __init__(self, conn_id, topic, *args, **kwargs): """ @@ -34,9 +38,13 @@ def __init__(self, conn_id, topic, *args, **kwargs): the subscribed topic """ self.topic = topic - self.hook = KafkaConsumerHook(conn_id, topic) + self.conn_id = conn_id super(KafkaSensor, self).__init__(*args, **kwargs) + @cached_property + def hook(self): + return KafkaConsumerHook(self.conn_id, self.topic) + def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', From 3334ba8b947e80d47f66400a3168092eabb8586d Mon Sep 17 00:00:00 2001 From: dan Date: Mon, 2 Mar 2020 15:47:10 -0500 Subject: [PATCH 024/162] Moved files to providers package. Externalized connection logic from init function --- airflow/contrib/hooks/kafka_consumer_hook.py | 71 ------------------- airflow/contrib/hooks/kafka_producer_hook.py | 58 --------------- airflow/contrib/sensors/kafka_sensor.py | 58 --------------- airflow/providers/apache/kafka/__init__.py | 24 +++---- .../apache/kafka/example_dags/__init__.py | 24 +++---- .../kafka/example_dags/example_kafka_dag.py | 22 +----- .../providers/apache/kafka/hooks/__init__.py | 24 +++---- .../apache/kafka/hooks/kafka_consumer_hook.py | 49 ++++--------- .../apache/kafka/hooks/kafka_producer_hook.py | 67 +++++------------ .../apache/kafka/sensors/__init__.py | 24 +++---- .../apache/kafka/sensors/kafka_sensor.py | 62 +++++++--------- 11 files changed, 96 insertions(+), 387 deletions(-) delete mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py delete mode 100644 airflow/contrib/hooks/kafka_producer_hook.py delete mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py deleted file mode 100644 index 1143e2ccfb6a7..0000000000000 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ /dev/null @@ -1,71 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaConsumer - - -class KafkaConsumerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - # Disable auto commit as the hook will commit right - # after polling. - conf['enable_auto_commit'] = False - - self.server = f"""{host}:{port}""" - self.consumer = KafkaConsumer( - self.topic, - bootstrap_servers=self.server, **conf) - - return self.consumer - - def get_messages(self, timeout_ms=50): - """ - Get all the messages haven't been consumed, it doesn't - block by default, then commit the offset. - :return: - A list of messages - """ - consumer = self.get_conn() - try: - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms) - - consumer.commit() - finally: - consumer.close() - return messages - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.consumer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py deleted file mode 100644 index 9c1ebb158afe5..0000000000000 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaProducer - - -class KafkaProducerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaProducerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - conf['enable_auto_commit'] = False - self.server = f"""{host}:{port}""" - self.producer = KafkaProducer( - bootstrap_servers=self.server, **conf) - - return self.producer - - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - producer = self.get_conn() - try: - future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, - timestamp_ms=timestamp_ms) - finally: - producer.close() - return future_record_metadata - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.produer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py deleted file mode 100644 index e51328cdd63a9..0000000000000 --- a/airflow/contrib/sensors/kafka_sensor.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import logging - -from werkzeug.utils import cached_property - -from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook -from airflow.sensors.base_sensor_operator import BaseSensorOperator -from airflow.utils.decorators import apply_defaults - - -class KafkaSensor(BaseSensorOperator): - """ - Consumes the Kafka message with the specific topic - """ - - templated_fields = ('topic',) - - @apply_defaults - def __init__(self, conn_id, topic, *args, **kwargs): - """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - :param conn_id: - the kafka broker connection whom this sensor - subscripts against. - :param topic: - the subscribed topic - """ - self.topic = topic - self.conn_id = conn_id - super(KafkaSensor, self).__init__(*args, **kwargs) - - @cached_property - def hook(self): - return KafkaConsumerHook(self.conn_id, self.topic) - - def poke(self, context): - logging.info( - 'Poking topic: %s, using hook: %s', - self.topic, self.hook) - - messages = self.hook.get_messages() - - logging.info( - 'Got messages during poking: %s', messages) - - return messages or False diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py index 217e5db960782..164f68b09175b 100644 --- a/airflow/providers/apache/kafka/__init__.py +++ b/airflow/providers/apache/kafka/__init__.py @@ -1,17 +1,11 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at # -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 # -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/airflow/providers/apache/kafka/example_dags/__init__.py b/airflow/providers/apache/kafka/example_dags/__init__.py index 217e5db960782..164f68b09175b 100644 --- a/airflow/providers/apache/kafka/example_dags/__init__.py +++ b/airflow/providers/apache/kafka/example_dags/__init__.py @@ -1,17 +1,11 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at # -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 # -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py index da333206b6423..e002b8d91a196 100644 --- a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py +++ b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py @@ -1,26 +1,8 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - +from airflow import DAG from datetime import timedelta -from airflow import DAG -from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor from airflow.utils.timezone import datetime +from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor DAG_ID = "example_kafka_dag" dag_start_date = datetime(2015, 6, 1, hour=20, tzinfo=None) diff --git a/airflow/providers/apache/kafka/hooks/__init__.py b/airflow/providers/apache/kafka/hooks/__init__.py index 217e5db960782..164f68b09175b 100644 --- a/airflow/providers/apache/kafka/hooks/__init__.py +++ b/airflow/providers/apache/kafka/hooks/__init__.py @@ -1,17 +1,11 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at # -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 # -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index df2576069e927..cbb2a4c963839 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -1,31 +1,21 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at # -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 # -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from airflow.hooks import base_hook as BaseHook from kafka import KafkaConsumer -from airflow.hooks.base_hook import BaseHook - class KafkaConsumerHook(BaseHook): - """ - KafkaConsumerHook Class. - """ + DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 @@ -40,13 +30,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self) -> KafkaConsumer: - """ - A Kafka Consumer object. - - :return: - A Kafka Consumer object. - """ + def get_conn(self): if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -61,12 +45,10 @@ def get_conn(self) -> KafkaConsumer: ) return self.consumer - def get_messages(self, timeout_ms=5000) -> dict: + def get_messages(self, timeout_ms=5000): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. - - :param timeout_ms: :return: A list of messages """ @@ -80,10 +62,7 @@ def get_messages(self, timeout_ms=5000) -> dict: def __repr__(self): """ - A pretty version of the connection string. - - :return: - A pretty version of the connection string. + Pretty the hook with the connection info """ connected = self.consumer is not None return ' KafkaProducer: - """ - Returns a Kafka Producer - - :return: - A Kafka Producer object. - """ + def get_conn(self): if not self._conn: - _conn = self.get_connection(self.conn_id) - service_options = _conn.extra_dejson - host = _conn.host or self.DEFAULT_HOST - port = _conn.port or self.DEFAULT_PORT + conn = self.get_connection(self.conn_id) + service_options = conn.extra_dejson + host = conn.host or self.DEFAULT_HOST + port = conn.port or self.DEFAULT_PORT self.server = f"""{host}:{port}""" self.consumer = KafkaProducer( @@ -58,16 +42,6 @@ def get_conn(self) -> KafkaProducer: return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - """ - Sends a message on the specified topic and partition. Keyed messages will be sent in order. - - :param topic: - :param value: - :param key: - :param partition: - :param timestamp_ms: - :return: - """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, @@ -78,12 +52,9 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - A pretty version of the connection string. - - :return: - A pretty version of the connection string. + Pretty the hook with the connection info """ - connected = self.producer is not None + connected = self.produer is not None return '' % \ (connected, self.server, self.topic) diff --git a/airflow/providers/apache/kafka/sensors/__init__.py b/airflow/providers/apache/kafka/sensors/__init__.py index 217e5db960782..164f68b09175b 100644 --- a/airflow/providers/apache/kafka/sensors/__init__.py +++ b/airflow/providers/apache/kafka/sensors/__init__.py @@ -1,17 +1,11 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at # -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 # -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 842643c60ba40..f7043a0294706 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -1,26 +1,21 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at # -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 # -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. -from cached_property import cached_property +import logging -from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook -from airflow.sensors.base_sensor_operator import BaseSensorOperator +from cached_property import cached_property from airflow.utils.decorators import apply_defaults +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook class KafkaSensor(BaseSensorOperator): @@ -37,14 +32,13 @@ class KafkaSensor(BaseSensorOperator): @apply_defaults def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs): """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. :param topic: - :param host: - :param port: - :param args: - :param kwargs: + the subscribed topic """ self.topic = topic self.host = host @@ -53,24 +47,18 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): - """ - Returns a Kafka Consumer Hook - """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): - """ - Checks to see if messages exist on this topic/partition. - - :param context: - :return: - """ - self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) messages = self.hook.get_messages() - if messages: - self.log.info('Got messages during poking: %s', str(messages)) + if messages is not {}: + logging.info( + 'Got messages during poking: %s', messages) return messages else: return False From e6ff80681bf2aad88df433b5da5480765f30937d Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 11 Mar 2020 09:06:55 -0400 Subject: [PATCH 025/162] Fixing Travis CI Linting errors --- airflow/providers/apache/kafka/__init__.py | 24 ++++++++++++------- .../providers/apache/kafka/hooks/__init__.py | 24 ++++++++++++------- .../apache/kafka/hooks/kafka_consumer_hook.py | 2 +- .../apache/kafka/hooks/kafka_producer_hook.py | 2 +- .../apache/kafka/sensors/kafka_sensor.py | 24 ++++++++++++------- 5 files changed, 47 insertions(+), 29 deletions(-) diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py index 164f68b09175b..217e5db960782 100644 --- a/airflow/providers/apache/kafka/__init__.py +++ b/airflow/providers/apache/kafka/__init__.py @@ -1,11 +1,17 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/airflow/providers/apache/kafka/hooks/__init__.py b/airflow/providers/apache/kafka/hooks/__init__.py index 164f68b09175b..217e5db960782 100644 --- a/airflow/providers/apache/kafka/hooks/__init__.py +++ b/airflow/providers/apache/kafka/hooks/__init__.py @@ -1,11 +1,17 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index cbb2a4c963839..968514398a20b 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -10,8 +10,8 @@ # See the License for the specific language governing permissions and # limitations under the License. -from airflow.hooks import base_hook as BaseHook from kafka import KafkaConsumer +from airflow.hooks.base_hook import BaseHook class KafkaConsumerHook(BaseHook): diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 4bc60e2a610fd..7d74110012529 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -10,8 +10,8 @@ # See the License for the specific language governing permissions and # limitations under the License. -from airflow.hooks import base_hook as BaseHook from kafka import KafkaProducer +from airflow.hooks.base_hook import BaseHook class KafkaProducerHook(BaseHook): diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index f7043a0294706..c5096d8734fcd 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -1,14 +1,20 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. import logging From 23b9ed1defc5d0d6bc26bd185cd88b20058b04de Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 11 Mar 2020 09:06:55 -0400 Subject: [PATCH 026/162] Fixing Travis CI Linting errors --- .../apache/kafka/example_dags/__init__.py | 24 ++++++++----- .../kafka/example_dags/example_kafka_dag.py | 18 ++++++++++ .../apache/kafka/hooks/kafka_consumer_hook.py | 25 ++++++++----- .../apache/kafka/hooks/kafka_producer_hook.py | 35 +++++++++++-------- .../apache/kafka/sensors/__init__.py | 24 ++++++++----- 5 files changed, 85 insertions(+), 41 deletions(-) diff --git a/airflow/providers/apache/kafka/example_dags/__init__.py b/airflow/providers/apache/kafka/example_dags/__init__.py index 164f68b09175b..217e5db960782 100644 --- a/airflow/providers/apache/kafka/example_dags/__init__.py +++ b/airflow/providers/apache/kafka/example_dags/__init__.py @@ -1,11 +1,17 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py index e002b8d91a196..eb89df6b7cca8 100644 --- a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py +++ b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py @@ -1,3 +1,21 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + from airflow import DAG from datetime import timedelta diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 968514398a20b..bb4895fab25fb 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -1,14 +1,21 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + from kafka import KafkaConsumer from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 7d74110012529..e7f70f2ef1aec 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -1,14 +1,20 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. from kafka import KafkaProducer from airflow.hooks.base_hook import BaseHook @@ -21,7 +27,8 @@ class KafkaProducerHook(BaseHook): def __init__(self, conn_id, topic): super(KafkaProducerHook, self).__init__(None) - self.conn = None + self.conn_id = conn_id + self._conn = None self.server = None self.consumer = None self.producer = None @@ -29,10 +36,10 @@ def __init__(self, conn_id, topic): def get_conn(self): if not self._conn: - conn = self.get_connection(self.conn_id) - service_options = conn.extra_dejson - host = conn.host or self.DEFAULT_HOST - port = conn.port or self.DEFAULT_PORT + _conn = self.get_connection(self.conn_id) + service_options = _conn.extra_dejson + host = _conn.host or self.DEFAULT_HOST + port = _conn.port or self.DEFAULT_PORT self.server = f"""{host}:{port}""" self.consumer = KafkaProducer( diff --git a/airflow/providers/apache/kafka/sensors/__init__.py b/airflow/providers/apache/kafka/sensors/__init__.py index 164f68b09175b..217e5db960782 100644 --- a/airflow/providers/apache/kafka/sensors/__init__.py +++ b/airflow/providers/apache/kafka/sensors/__init__.py @@ -1,11 +1,17 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. From 136dc9b441fcf4c417429ddb11965766a6737177 Mon Sep 17 00:00:00 2001 From: dan Date: Thu, 30 Apr 2020 09:57:14 -0400 Subject: [PATCH 027/162] Fixed isort issue --- .../providers/apache/kafka/example_dags/example_kafka_dag.py | 4 ++-- airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py | 1 + airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 1 + airflow/providers/apache/kafka/sensors/kafka_sensor.py | 5 +++-- 4 files changed, 7 insertions(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py index eb89df6b7cca8..da333206b6423 100644 --- a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py +++ b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py @@ -16,11 +16,11 @@ # specific language governing permissions and limitations # under the License. -from airflow import DAG from datetime import timedelta -from airflow.utils.timezone import datetime +from airflow import DAG from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor +from airflow.utils.timezone import datetime DAG_ID = "example_kafka_dag" dag_start_date = datetime(2015, 6, 1, hour=20, tzinfo=None) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index bb4895fab25fb..c5208508a6e3d 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -18,6 +18,7 @@ from kafka import KafkaConsumer + from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index e7f70f2ef1aec..f7410ee097fb7 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -17,6 +17,7 @@ # under the License. from kafka import KafkaProducer + from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index c5096d8734fcd..8649f876407ec 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -19,9 +19,10 @@ import logging from cached_property import cached_property -from airflow.utils.decorators import apply_defaults -from airflow.sensors.base_sensor_operator import BaseSensorOperator + from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class KafkaSensor(BaseSensorOperator): From 8197a59046c9392ca861284ccb3325876ab6969b Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:16:08 -0400 Subject: [PATCH 028/162] produer --> producer --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index f7410ee097fb7..5272de1cbf1bc 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -62,7 +62,7 @@ def __repr__(self): """ Pretty the hook with the connection info """ - connected = self.produer is not None + connected = self.producer is not None return '' % \ (connected, self.server, self.topic) From 788586e688e91e184fb640255de173454261fc5f Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:36:16 -0400 Subject: [PATCH 029/162] Added pylint mandated doc strings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 2 ++ .../apache/kafka/hooks/kafka_producer_hook.py | 11 +++++++++++ .../providers/apache/kafka/sensors/kafka_sensor.py | 7 ++++++- 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index c5208508a6e3d..d7cd9b2d175cb 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -23,7 +23,9 @@ class KafkaConsumerHook(BaseHook): + """ + """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 5272de1cbf1bc..35232c253b819 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -22,7 +22,9 @@ class KafkaProducerHook(BaseHook): + """ + """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 @@ -50,6 +52,15 @@ def get_conn(self): return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + """ + + :param topic: + :param value: + :param key: + :param partition: + :param timestamp_ms: + :return: + """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 8649f876407ec..e4c33f5fa16f8 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -54,12 +54,17 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): + """ + + :return: + KafkaConsumerHook + """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', - self.topic, self.hook) + str(self.topic), str(self.hook)) messages = self.hook.get_messages() From 10531c059fa44a4b4a11b68c2ac48b462fe8a6b8 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 10:06:05 -0400 Subject: [PATCH 030/162] Fixing more pylint issues --- airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py | 2 +- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index d7cd9b2d175cb..025ebc1077668 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -24,7 +24,7 @@ class KafkaConsumerHook(BaseHook): """ - + KafkaConsumerHook Class. """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 35232c253b819..affa37ef3ac05 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -23,7 +23,7 @@ class KafkaProducerHook(BaseHook): """ - + KafkaProducerHook Class. """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index e4c33f5fa16f8..250656398ac4e 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -68,9 +68,9 @@ def poke(self, context): messages = self.hook.get_messages() - if messages is not {}: + if not messages: logging.info( - 'Got messages during poking: %s', messages) + 'Got messages during poking: %s', str(messages)) return messages else: return False From ba8661127acb5cc17bfd8287adf4d8b0ccebdc11 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 031/162] Tried to fix docstrings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 17 +++++++++++--- .../apache/kafka/hooks/kafka_producer_hook.py | 17 +++++++++++--- .../apache/kafka/sensors/kafka_sensor.py | 22 +++++++++++++------ 3 files changed, 43 insertions(+), 13 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 025ebc1077668..df2576069e927 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -40,7 +40,13 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self): + def get_conn(self) -> KafkaConsumer: + """ + A Kafka Consumer object. + + :return: + A Kafka Consumer object. + """ if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -55,10 +61,12 @@ def get_conn(self): ) return self.consumer - def get_messages(self, timeout_ms=5000): + def get_messages(self, timeout_ms=5000) -> dict: """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. + + :param timeout_ms: :return: A list of messages """ @@ -72,7 +80,10 @@ def get_messages(self, timeout_ms=5000): def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.consumer is not None return ' KafkaProducer: + """ + Returns a Kafka Producer + + :return: + A Kafka Producer object. + """ if not self._conn: _conn = self.get_connection(self.conn_id) service_options = _conn.extra_dejson @@ -51,8 +58,9 @@ def get_conn(self): ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: """ + Sends a message on the specified topic and partition. Keyed messages will be sent in order. :param topic: :param value: @@ -71,7 +79,10 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.producer is not None return ' Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 032/162] Tried to fix docstrings --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 6bef4e4b56381..9919505b4a24d 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -58,7 +58,7 @@ def get_conn(self) -> KafkaProducer: ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): """ Sends a message on the specified topic and partition. Keyed messages will be sent in order. From 0ec3ec79b2534fa60ffe48aac585a2d2c521cfe6 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 9 May 2020 23:41:58 +0100 Subject: [PATCH 033/162] Apply suggestions from code review --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 50ca55079d9ac..60b95989b64cb 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -56,10 +56,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): """ - Returns a Kafka Consumer Hook - - :return: - KafkaConsumerHook + Returns a Kafka Consumer Hook """ return KafkaConsumerHook(self.topic, self.host, self.port) From da9daf15ccebe0718ece271303bcf29a8c56f239 Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:28 -0400 Subject: [PATCH 034/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 60b95989b64cb..7108f84207bcd 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -74,8 +74,7 @@ def poke(self, context): messages = self.hook.get_messages() if messages: - logging.info( - 'Got messages during poking: %s', str(messages)) + self.log.info('Got messages during poking: %s', str(messages)) return messages else: return False From 8688e6a9174c432e0bf806791fe0f820f03b59db Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:46 -0400 Subject: [PATCH 035/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 7108f84207bcd..bb27e2f5adc73 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -67,9 +67,7 @@ def poke(self, context): :param context: :return: """ - logging.info( - 'Poking topic: %s, using hook: %s', - str(self.topic), str(self.hook)) + self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) messages = self.hook.get_messages() From b600971bb8204e1cb6adba233dbb65fadf1d8144 Mon Sep 17 00:00:00 2001 From: dan Date: Sat, 23 May 2020 14:46:59 -0400 Subject: [PATCH 036/162] Removed unused imports --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 1 - airflow/providers/apache/kafka/sensors/kafka_sensor.py | 2 -- 2 files changed, 3 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 9919505b4a24d..f6d9fd2428ec0 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -17,7 +17,6 @@ # under the License. from kafka import KafkaProducer -from kafka.producer.future import FutureRecordMetadata from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index bb27e2f5adc73..842643c60ba40 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -16,8 +16,6 @@ # specific language governing permissions and limitations # under the License. -import logging - from cached_property import cached_property from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook From 2789458ed8bfd9f22f8732567358e044e43aa4a0 Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 12 Feb 2020 13:51:03 -0500 Subject: [PATCH 037/162] Add KafkaConsumerHook, KafkaProduerHook and KafkaSensor MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Dan Ferguson dferguson992@gmail.com Co-authored-by: Hanan Shteingart Co-authored-by: YuanfΞi Zhu --- airflow/contrib/hooks/kafka_consumer_hook.py | 69 ++++++++++++++++++++ airflow/contrib/hooks/kafka_producer_hook.py | 56 ++++++++++++++++ airflow/contrib/sensors/kafka_sensor.py | 50 ++++++++++++++ 3 files changed, 175 insertions(+) create mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py create mode 100644 airflow/contrib/hooks/kafka_producer_hook.py create mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py new file mode 100644 index 0000000000000..335c267b7837f --- /dev/null +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -0,0 +1,69 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaConsumer + + +class KafkaConsumerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + # Disable auto commit as the hook will commit right + # after polling. + conf['enable_auto_commit'] = False + + self.server = '{host}:{port}'.format(**locals()) + self.consumer = KafkaConsumer( + self.topic, + bootstrap_servers=self.server, **conf) + + return self.consumer + + def get_messages(self): + """ + Get all the messages haven't been consumed, it doesn't + block by default, then commit the offset. + :return: + A list of messages + """ + consumer = self.get_conn() + + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms=50) + + consumer.commit() + return messages + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.consumer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py new file mode 100644 index 0000000000000..ffa5d4693e9c2 --- /dev/null +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -0,0 +1,56 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaProducer + + +class KafkaProducerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + conf['enable_auto_commit'] = False + self.server = '{host}:{port}'.format(**locals()) + self.producer = KafkaProducer( + bootstrap_servers=self.server, **conf) + + return self.producer + + def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): + producer = self.get_pub() + future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, + timestamp_ms=timestamp_ms) + producer.close() + return future_record_metadata + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.produer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py new file mode 100644 index 0000000000000..738e63c76c730 --- /dev/null +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -0,0 +1,50 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import logging + +from src.kafka_hook import KafkaConsumerHook +from airflow.operators.sensors import BaseSensorOperator +from airflow.utils import apply_defaults + + +class KafkaSensor(BaseSensorOperator): + """ + Consumes the Kafka message with the specific topic + """ + + @apply_defaults + def __init__(self, conn_id, topic, *args, **kwargs): + """ + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. + :param topic: + the subscribed topic + """ + self.topic = topic + self.hook = KafkaConsumerHook(conn_id, topic) + super(KafkaSensor, self).__init__(*args, **kwargs) + + def poke(self, context): + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) + + messages = self.hook.get_messages() + + logging.info( + 'Got messages during poking: %s', messages) + + return messages or False From d5673a282a9a6abf53bafcbb576af2b472986e85 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 18 Feb 2020 09:05:42 -0500 Subject: [PATCH 038/162] Modifying files in accordance with peer review --- airflow/contrib/hooks/kafka_consumer_hook.py | 24 +++++++++++--------- airflow/contrib/hooks/kafka_producer_hook.py | 24 +++++++++++--------- airflow/contrib/sensors/kafka_sensor.py | 16 +++++++++---- 3 files changed, 38 insertions(+), 26 deletions(-) diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py index 335c267b7837f..1143e2ccfb6a7 100644 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -16,8 +16,8 @@ class KafkaConsumerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): super(KafkaConsumerHook, self).__init__(None) @@ -29,21 +29,21 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT # Disable auto commit as the hook will commit right # after polling. conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.consumer = KafkaConsumer( self.topic, bootstrap_servers=self.server, **conf) return self.consumer - def get_messages(self): + def get_messages(self, timeout_ms=50): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. @@ -51,12 +51,14 @@ def get_messages(self): A list of messages """ consumer = self.get_conn() + try: + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms) - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms=50) - - consumer.commit() + consumer.commit() + finally: + consumer.close() return messages def __repr__(self): diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py index ffa5d4693e9c2..9c1ebb158afe5 100644 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -16,11 +16,11 @@ class KafkaProducerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) + super(KafkaProducerHook, self).__init__(None) self.conn = self.get_connection(conn_id) self.server = None self.consumer = None @@ -29,21 +29,23 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.producer = KafkaProducer( bootstrap_servers=self.server, **conf) return self.producer - def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): - producer = self.get_pub() - future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, - timestamp_ms=timestamp_ms) - producer.close() + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + producer = self.get_conn() + try: + future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, + timestamp_ms=timestamp_ms) + finally: + producer.close() return future_record_metadata def __repr__(self): diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py index 738e63c76c730..e51328cdd63a9 100644 --- a/airflow/contrib/sensors/kafka_sensor.py +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -12,9 +12,11 @@ import logging -from src.kafka_hook import KafkaConsumerHook -from airflow.operators.sensors import BaseSensorOperator -from airflow.utils import apply_defaults +from werkzeug.utils import cached_property + +from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class KafkaSensor(BaseSensorOperator): @@ -22,6 +24,8 @@ class KafkaSensor(BaseSensorOperator): Consumes the Kafka message with the specific topic """ + templated_fields = ('topic',) + @apply_defaults def __init__(self, conn_id, topic, *args, **kwargs): """ @@ -34,9 +38,13 @@ def __init__(self, conn_id, topic, *args, **kwargs): the subscribed topic """ self.topic = topic - self.hook = KafkaConsumerHook(conn_id, topic) + self.conn_id = conn_id super(KafkaSensor, self).__init__(*args, **kwargs) + @cached_property + def hook(self): + return KafkaConsumerHook(self.conn_id, self.topic) + def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', From 30982c79e21bb5f11545527868ecb51d4716c5aa Mon Sep 17 00:00:00 2001 From: dan Date: Mon, 2 Mar 2020 15:47:10 -0500 Subject: [PATCH 039/162] Moved files to providers package. Externalized connection logic from init function --- airflow/contrib/hooks/kafka_consumer_hook.py | 71 ------------------- airflow/contrib/hooks/kafka_producer_hook.py | 58 --------------- airflow/contrib/sensors/kafka_sensor.py | 58 --------------- airflow/providers/apache/kafka/__init__.py | 14 ++++ .../apache/kafka/example_dags/__init__.py | 14 ++++ .../kafka/example_dags/example_kafka_dag.py | 9 +++ .../providers/apache/kafka/hooks/__init__.py | 14 ++++ .../apache/kafka/hooks/kafka_consumer_hook.py | 49 ++++--------- .../apache/kafka/hooks/kafka_producer_hook.py | 67 +++++------------ .../apache/kafka/sensors/__init__.py | 14 ++++ .../apache/kafka/sensors/kafka_sensor.py | 62 +++++++--------- 11 files changed, 123 insertions(+), 307 deletions(-) delete mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py delete mode 100644 airflow/contrib/hooks/kafka_producer_hook.py delete mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py deleted file mode 100644 index 1143e2ccfb6a7..0000000000000 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ /dev/null @@ -1,71 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaConsumer - - -class KafkaConsumerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - # Disable auto commit as the hook will commit right - # after polling. - conf['enable_auto_commit'] = False - - self.server = f"""{host}:{port}""" - self.consumer = KafkaConsumer( - self.topic, - bootstrap_servers=self.server, **conf) - - return self.consumer - - def get_messages(self, timeout_ms=50): - """ - Get all the messages haven't been consumed, it doesn't - block by default, then commit the offset. - :return: - A list of messages - """ - consumer = self.get_conn() - try: - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms) - - consumer.commit() - finally: - consumer.close() - return messages - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.consumer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py deleted file mode 100644 index 9c1ebb158afe5..0000000000000 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaProducer - - -class KafkaProducerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaProducerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - conf['enable_auto_commit'] = False - self.server = f"""{host}:{port}""" - self.producer = KafkaProducer( - bootstrap_servers=self.server, **conf) - - return self.producer - - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - producer = self.get_conn() - try: - future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, - timestamp_ms=timestamp_ms) - finally: - producer.close() - return future_record_metadata - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.produer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py deleted file mode 100644 index e51328cdd63a9..0000000000000 --- a/airflow/contrib/sensors/kafka_sensor.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import logging - -from werkzeug.utils import cached_property - -from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook -from airflow.sensors.base_sensor_operator import BaseSensorOperator -from airflow.utils.decorators import apply_defaults - - -class KafkaSensor(BaseSensorOperator): - """ - Consumes the Kafka message with the specific topic - """ - - templated_fields = ('topic',) - - @apply_defaults - def __init__(self, conn_id, topic, *args, **kwargs): - """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - :param conn_id: - the kafka broker connection whom this sensor - subscripts against. - :param topic: - the subscribed topic - """ - self.topic = topic - self.conn_id = conn_id - super(KafkaSensor, self).__init__(*args, **kwargs) - - @cached_property - def hook(self): - return KafkaConsumerHook(self.conn_id, self.topic) - - def poke(self, context): - logging.info( - 'Poking topic: %s, using hook: %s', - self.topic, self.hook) - - messages = self.hook.get_messages() - - logging.info( - 'Got messages during poking: %s', messages) - - return messages or False diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py index 217e5db960782..e136811d94d06 100644 --- a/airflow/providers/apache/kafka/__init__.py +++ b/airflow/providers/apache/kafka/__init__.py @@ -1,3 +1,4 @@ +<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -15,3 +16,16 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +======= +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +>>>>>>> Moved files to providers package. Externalized connection logic from init function diff --git a/airflow/providers/apache/kafka/example_dags/__init__.py b/airflow/providers/apache/kafka/example_dags/__init__.py index 217e5db960782..e136811d94d06 100644 --- a/airflow/providers/apache/kafka/example_dags/__init__.py +++ b/airflow/providers/apache/kafka/example_dags/__init__.py @@ -1,3 +1,4 @@ +<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -15,3 +16,16 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +======= +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +>>>>>>> Moved files to providers package. Externalized connection logic from init function diff --git a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py index da333206b6423..8baa81c7a1962 100644 --- a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py +++ b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py @@ -1,3 +1,4 @@ +<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -21,6 +22,14 @@ from airflow import DAG from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor from airflow.utils.timezone import datetime +======= + +from airflow import DAG +from datetime import timedelta + +from airflow.utils.timezone import datetime +from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor +>>>>>>> Moved files to providers package. Externalized connection logic from init function DAG_ID = "example_kafka_dag" dag_start_date = datetime(2015, 6, 1, hour=20, tzinfo=None) diff --git a/airflow/providers/apache/kafka/hooks/__init__.py b/airflow/providers/apache/kafka/hooks/__init__.py index 217e5db960782..e136811d94d06 100644 --- a/airflow/providers/apache/kafka/hooks/__init__.py +++ b/airflow/providers/apache/kafka/hooks/__init__.py @@ -1,3 +1,4 @@ +<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -15,3 +16,16 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +======= +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +>>>>>>> Moved files to providers package. Externalized connection logic from init function diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index df2576069e927..cbb2a4c963839 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -1,31 +1,21 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at # -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 # -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from airflow.hooks import base_hook as BaseHook from kafka import KafkaConsumer -from airflow.hooks.base_hook import BaseHook - class KafkaConsumerHook(BaseHook): - """ - KafkaConsumerHook Class. - """ + DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 @@ -40,13 +30,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self) -> KafkaConsumer: - """ - A Kafka Consumer object. - - :return: - A Kafka Consumer object. - """ + def get_conn(self): if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -61,12 +45,10 @@ def get_conn(self) -> KafkaConsumer: ) return self.consumer - def get_messages(self, timeout_ms=5000) -> dict: + def get_messages(self, timeout_ms=5000): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. - - :param timeout_ms: :return: A list of messages """ @@ -80,10 +62,7 @@ def get_messages(self, timeout_ms=5000) -> dict: def __repr__(self): """ - A pretty version of the connection string. - - :return: - A pretty version of the connection string. + Pretty the hook with the connection info """ connected = self.consumer is not None return ' KafkaProducer: - """ - Returns a Kafka Producer - - :return: - A Kafka Producer object. - """ + def get_conn(self): if not self._conn: - _conn = self.get_connection(self.conn_id) - service_options = _conn.extra_dejson - host = _conn.host or self.DEFAULT_HOST - port = _conn.port or self.DEFAULT_PORT + conn = self.get_connection(self.conn_id) + service_options = conn.extra_dejson + host = conn.host or self.DEFAULT_HOST + port = conn.port or self.DEFAULT_PORT self.server = f"""{host}:{port}""" self.consumer = KafkaProducer( @@ -58,16 +42,6 @@ def get_conn(self) -> KafkaProducer: return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - """ - Sends a message on the specified topic and partition. Keyed messages will be sent in order. - - :param topic: - :param value: - :param key: - :param partition: - :param timestamp_ms: - :return: - """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, @@ -78,12 +52,9 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - A pretty version of the connection string. - - :return: - A pretty version of the connection string. + Pretty the hook with the connection info """ - connected = self.producer is not None + connected = self.produer is not None return '' % \ (connected, self.server, self.topic) diff --git a/airflow/providers/apache/kafka/sensors/__init__.py b/airflow/providers/apache/kafka/sensors/__init__.py index 217e5db960782..e136811d94d06 100644 --- a/airflow/providers/apache/kafka/sensors/__init__.py +++ b/airflow/providers/apache/kafka/sensors/__init__.py @@ -1,3 +1,4 @@ +<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -15,3 +16,16 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +======= +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +>>>>>>> Moved files to providers package. Externalized connection logic from init function diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 842643c60ba40..f7043a0294706 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -1,26 +1,21 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at # -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 # -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. -from cached_property import cached_property +import logging -from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook -from airflow.sensors.base_sensor_operator import BaseSensorOperator +from cached_property import cached_property from airflow.utils.decorators import apply_defaults +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook class KafkaSensor(BaseSensorOperator): @@ -37,14 +32,13 @@ class KafkaSensor(BaseSensorOperator): @apply_defaults def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs): """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. :param topic: - :param host: - :param port: - :param args: - :param kwargs: + the subscribed topic """ self.topic = topic self.host = host @@ -53,24 +47,18 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): - """ - Returns a Kafka Consumer Hook - """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): - """ - Checks to see if messages exist on this topic/partition. - - :param context: - :return: - """ - self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) messages = self.hook.get_messages() - if messages: - self.log.info('Got messages during poking: %s', str(messages)) + if messages is not {}: + logging.info( + 'Got messages during poking: %s', messages) return messages else: return False From 2847ce0ee41bcf3eab8818cab9cae9471affa24a Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 11 Mar 2020 09:06:55 -0400 Subject: [PATCH 040/162] Fixing Travis CI Linting errors --- airflow/providers/apache/kafka/__init__.py | 22 ++++++++++++++++- .../providers/apache/kafka/hooks/__init__.py | 22 ++++++++++++++++- .../apache/kafka/hooks/kafka_consumer_hook.py | 2 +- .../apache/kafka/hooks/kafka_producer_hook.py | 2 +- .../apache/kafka/sensors/kafka_sensor.py | 24 ++++++++++++------- 5 files changed, 59 insertions(+), 13 deletions(-) diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py index e136811d94d06..15388643f828a 100644 --- a/airflow/providers/apache/kafka/__init__.py +++ b/airflow/providers/apache/kafka/__init__.py @@ -1,4 +1,5 @@ <<<<<<< HEAD +<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -20,12 +21,31 @@ # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. # You may obtain a copy of the License at +======= +>>>>>>> Fixing Travis CI Linting errors # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # +<<<<<<< HEAD # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. >>>>>>> Moved files to providers package. Externalized connection logic from init function +======= +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +>>>>>>> Fixing Travis CI Linting errors diff --git a/airflow/providers/apache/kafka/hooks/__init__.py b/airflow/providers/apache/kafka/hooks/__init__.py index e136811d94d06..15388643f828a 100644 --- a/airflow/providers/apache/kafka/hooks/__init__.py +++ b/airflow/providers/apache/kafka/hooks/__init__.py @@ -1,4 +1,5 @@ <<<<<<< HEAD +<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -20,12 +21,31 @@ # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. # You may obtain a copy of the License at +======= +>>>>>>> Fixing Travis CI Linting errors # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # +<<<<<<< HEAD # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. >>>>>>> Moved files to providers package. Externalized connection logic from init function +======= +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +>>>>>>> Fixing Travis CI Linting errors diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index cbb2a4c963839..968514398a20b 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -10,8 +10,8 @@ # See the License for the specific language governing permissions and # limitations under the License. -from airflow.hooks import base_hook as BaseHook from kafka import KafkaConsumer +from airflow.hooks.base_hook import BaseHook class KafkaConsumerHook(BaseHook): diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 4bc60e2a610fd..7d74110012529 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -10,8 +10,8 @@ # See the License for the specific language governing permissions and # limitations under the License. -from airflow.hooks import base_hook as BaseHook from kafka import KafkaProducer +from airflow.hooks.base_hook import BaseHook class KafkaProducerHook(BaseHook): diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index f7043a0294706..c5096d8734fcd 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -1,14 +1,20 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. import logging From 4327d3da511bebc5a7d79443ffb6aca2992ea161 Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 11 Mar 2020 09:06:55 -0400 Subject: [PATCH 041/162] Fixing Travis CI Linting errors --- airflow/providers/apache/kafka/__init__.py | 34 ------------ .../apache/kafka/example_dags/__init__.py | 14 ----- .../kafka/example_dags/example_kafka_dag.py | 54 ------------------- .../providers/apache/kafka/hooks/__init__.py | 34 ------------ .../apache/kafka/hooks/kafka_consumer_hook.py | 25 +++++---- .../apache/kafka/hooks/kafka_producer_hook.py | 35 +++++++----- .../apache/kafka/sensors/__init__.py | 14 ----- 7 files changed, 37 insertions(+), 173 deletions(-) delete mode 100644 airflow/providers/apache/kafka/example_dags/example_kafka_dag.py diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py index 15388643f828a..217e5db960782 100644 --- a/airflow/providers/apache/kafka/__init__.py +++ b/airflow/providers/apache/kafka/__init__.py @@ -1,5 +1,3 @@ -<<<<<<< HEAD -<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -17,35 +15,3 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -======= -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -======= ->>>>>>> Fixing Travis CI Linting errors -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -<<<<<<< HEAD -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. ->>>>>>> Moved files to providers package. Externalized connection logic from init function -======= -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. ->>>>>>> Fixing Travis CI Linting errors diff --git a/airflow/providers/apache/kafka/example_dags/__init__.py b/airflow/providers/apache/kafka/example_dags/__init__.py index e136811d94d06..217e5db960782 100644 --- a/airflow/providers/apache/kafka/example_dags/__init__.py +++ b/airflow/providers/apache/kafka/example_dags/__init__.py @@ -1,4 +1,3 @@ -<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -16,16 +15,3 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -======= -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. ->>>>>>> Moved files to providers package. Externalized connection logic from init function diff --git a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py deleted file mode 100644 index 8baa81c7a1962..0000000000000 --- a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py +++ /dev/null @@ -1,54 +0,0 @@ -<<<<<<< HEAD -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -from datetime import timedelta - -from airflow import DAG -from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor -from airflow.utils.timezone import datetime -======= - -from airflow import DAG -from datetime import timedelta - -from airflow.utils.timezone import datetime -from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor ->>>>>>> Moved files to providers package. Externalized connection logic from init function - -DAG_ID = "example_kafka_dag" -dag_start_date = datetime(2015, 6, 1, hour=20, tzinfo=None) -default_args = { - 'owner': '@Ferg_In', - 'depends_on_past': False, - 'start_date': dag_start_date, - 'email': ['dferguson992@gmail.com'], - 'provide_context': True, - 'retries': 3, - 'retry_delay': timedelta(minutes=5), -} - -with DAG(dag_id=DAG_ID, default_args=default_args, schedule_interval=None, - max_active_runs=1, concurrency=4, catchup=False) as dag: - - sensor = KafkaSensor( - task_id='trigger', - topic='', - host='', - port='' - ) diff --git a/airflow/providers/apache/kafka/hooks/__init__.py b/airflow/providers/apache/kafka/hooks/__init__.py index 15388643f828a..217e5db960782 100644 --- a/airflow/providers/apache/kafka/hooks/__init__.py +++ b/airflow/providers/apache/kafka/hooks/__init__.py @@ -1,5 +1,3 @@ -<<<<<<< HEAD -<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -17,35 +15,3 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -======= -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -======= ->>>>>>> Fixing Travis CI Linting errors -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -<<<<<<< HEAD -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. ->>>>>>> Moved files to providers package. Externalized connection logic from init function -======= -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. ->>>>>>> Fixing Travis CI Linting errors diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 968514398a20b..bb4895fab25fb 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -1,14 +1,21 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + from kafka import KafkaConsumer from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 7d74110012529..e7f70f2ef1aec 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -1,14 +1,20 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. from kafka import KafkaProducer from airflow.hooks.base_hook import BaseHook @@ -21,7 +27,8 @@ class KafkaProducerHook(BaseHook): def __init__(self, conn_id, topic): super(KafkaProducerHook, self).__init__(None) - self.conn = None + self.conn_id = conn_id + self._conn = None self.server = None self.consumer = None self.producer = None @@ -29,10 +36,10 @@ def __init__(self, conn_id, topic): def get_conn(self): if not self._conn: - conn = self.get_connection(self.conn_id) - service_options = conn.extra_dejson - host = conn.host or self.DEFAULT_HOST - port = conn.port or self.DEFAULT_PORT + _conn = self.get_connection(self.conn_id) + service_options = _conn.extra_dejson + host = _conn.host or self.DEFAULT_HOST + port = _conn.port or self.DEFAULT_PORT self.server = f"""{host}:{port}""" self.consumer = KafkaProducer( diff --git a/airflow/providers/apache/kafka/sensors/__init__.py b/airflow/providers/apache/kafka/sensors/__init__.py index e136811d94d06..217e5db960782 100644 --- a/airflow/providers/apache/kafka/sensors/__init__.py +++ b/airflow/providers/apache/kafka/sensors/__init__.py @@ -1,4 +1,3 @@ -<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -16,16 +15,3 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -======= -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. ->>>>>>> Moved files to providers package. Externalized connection logic from init function From b858e8b8649936e0eb7a3e678cde6a7aac17df65 Mon Sep 17 00:00:00 2001 From: dan Date: Thu, 30 Apr 2020 09:57:14 -0400 Subject: [PATCH 042/162] Fixed isort issue --- airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py | 1 + airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 1 + airflow/providers/apache/kafka/sensors/kafka_sensor.py | 5 +++-- .../providers/docker/example_dags/example_docker_swarm.py | 3 +++ 4 files changed, 8 insertions(+), 2 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index bb4895fab25fb..c5208508a6e3d 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -18,6 +18,7 @@ from kafka import KafkaConsumer + from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index e7f70f2ef1aec..f7410ee097fb7 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -17,6 +17,7 @@ # under the License. from kafka import KafkaProducer + from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index c5096d8734fcd..8649f876407ec 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -19,9 +19,10 @@ import logging from cached_property import cached_property -from airflow.utils.decorators import apply_defaults -from airflow.sensors.base_sensor_operator import BaseSensorOperator + from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class KafkaSensor(BaseSensorOperator): diff --git a/airflow/providers/docker/example_dags/example_docker_swarm.py b/airflow/providers/docker/example_dags/example_docker_swarm.py index 7dc056edad0bb..3930482ec1fa1 100644 --- a/airflow/providers/docker/example_dags/example_docker_swarm.py +++ b/airflow/providers/docker/example_dags/example_docker_swarm.py @@ -15,11 +15,14 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. + from datetime import timedelta from airflow import DAG +from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor from airflow.providers.docker.operators.docker_swarm import DockerSwarmOperator from airflow.utils.dates import days_ago +from airflow.utils.timezone import datetime default_args = { 'owner': 'airflow', From deba782d8b604055654261db313586aabdcdba6c Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:16:08 -0400 Subject: [PATCH 043/162] produer --> producer --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index f7410ee097fb7..5272de1cbf1bc 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -62,7 +62,7 @@ def __repr__(self): """ Pretty the hook with the connection info """ - connected = self.produer is not None + connected = self.producer is not None return '' % \ (connected, self.server, self.topic) From 5c5c393422bec561ccb0f7a18273130231ee52c7 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:36:16 -0400 Subject: [PATCH 044/162] Added pylint mandated doc strings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 2 ++ .../apache/kafka/hooks/kafka_producer_hook.py | 11 +++++++++++ .../providers/apache/kafka/sensors/kafka_sensor.py | 7 ++++++- 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index c5208508a6e3d..d7cd9b2d175cb 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -23,7 +23,9 @@ class KafkaConsumerHook(BaseHook): + """ + """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 5272de1cbf1bc..35232c253b819 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -22,7 +22,9 @@ class KafkaProducerHook(BaseHook): + """ + """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 @@ -50,6 +52,15 @@ def get_conn(self): return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + """ + + :param topic: + :param value: + :param key: + :param partition: + :param timestamp_ms: + :return: + """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 8649f876407ec..e4c33f5fa16f8 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -54,12 +54,17 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): + """ + + :return: + KafkaConsumerHook + """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', - self.topic, self.hook) + str(self.topic), str(self.hook)) messages = self.hook.get_messages() From 8a70321eff5e176660df9cf33d69b2d6d8cdb11d Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 10:06:05 -0400 Subject: [PATCH 045/162] Fixing more pylint issues --- airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py | 2 +- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index d7cd9b2d175cb..025ebc1077668 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -24,7 +24,7 @@ class KafkaConsumerHook(BaseHook): """ - + KafkaConsumerHook Class. """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 35232c253b819..affa37ef3ac05 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -23,7 +23,7 @@ class KafkaProducerHook(BaseHook): """ - + KafkaProducerHook Class. """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index e4c33f5fa16f8..250656398ac4e 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -68,9 +68,9 @@ def poke(self, context): messages = self.hook.get_messages() - if messages is not {}: + if not messages: logging.info( - 'Got messages during poking: %s', messages) + 'Got messages during poking: %s', str(messages)) return messages else: return False From f71fa3805761571b1847071ad47ae55a5cce8ab6 Mon Sep 17 00:00:00 2001 From: dan Date: Sat, 23 May 2020 14:53:06 -0400 Subject: [PATCH 046/162] Removed unused imports --- docs/operators-and-hooks-ref.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/operators-and-hooks-ref.rst b/docs/operators-and-hooks-ref.rst index 1b5128b65b8af..6ff38f4aa5e77 100644 --- a/docs/operators-and-hooks-ref.rst +++ b/docs/operators-and-hooks-ref.rst @@ -145,7 +145,7 @@ Foundation. * - `Apache Kafka `__ - - :mod:`airflow.providers.apache.kafka.hooks.kafka_producer_hook`, - - :mod:`airflow.providers.apache.kafka.hooks.kafka_consumer_hook` + :mod:`airflow.providers.apache.kafka.hooks.kafka_consumer_hook` - :mod:`airflow.providers.apache.kafka.sensors.kafka_sensor` * - `Apache Hive `__ From b16b402a2a5f2568d33a0d020b832eaed12c69c4 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 047/162] Tried to fix docstrings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 17 +++++++++++--- .../apache/kafka/hooks/kafka_producer_hook.py | 17 +++++++++++--- .../apache/kafka/sensors/kafka_sensor.py | 22 +++++++++++++------ 3 files changed, 43 insertions(+), 13 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 025ebc1077668..df2576069e927 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -40,7 +40,13 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self): + def get_conn(self) -> KafkaConsumer: + """ + A Kafka Consumer object. + + :return: + A Kafka Consumer object. + """ if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -55,10 +61,12 @@ def get_conn(self): ) return self.consumer - def get_messages(self, timeout_ms=5000): + def get_messages(self, timeout_ms=5000) -> dict: """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. + + :param timeout_ms: :return: A list of messages """ @@ -72,7 +80,10 @@ def get_messages(self, timeout_ms=5000): def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.consumer is not None return ' KafkaProducer: + """ + Returns a Kafka Producer + + :return: + A Kafka Producer object. + """ if not self._conn: _conn = self.get_connection(self.conn_id) service_options = _conn.extra_dejson @@ -51,8 +58,9 @@ def get_conn(self): ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: """ + Sends a message on the specified topic and partition. Keyed messages will be sent in order. :param topic: :param value: @@ -71,7 +79,10 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.producer is not None return ' Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 048/162] Tried to fix docstrings --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 6bef4e4b56381..9919505b4a24d 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -58,7 +58,7 @@ def get_conn(self) -> KafkaProducer: ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): """ Sends a message on the specified topic and partition. Keyed messages will be sent in order. From b8239d7c5ba4d0a79b5448832b8b3b2e477e3cc2 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 9 May 2020 23:41:58 +0100 Subject: [PATCH 049/162] Apply suggestions from code review --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 50ca55079d9ac..60b95989b64cb 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -56,10 +56,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): """ - Returns a Kafka Consumer Hook - - :return: - KafkaConsumerHook + Returns a Kafka Consumer Hook """ return KafkaConsumerHook(self.topic, self.host, self.port) From 57d08d721e05de727702a2cf9be9aa01be6afeab Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:28 -0400 Subject: [PATCH 050/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 60b95989b64cb..7108f84207bcd 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -74,8 +74,7 @@ def poke(self, context): messages = self.hook.get_messages() if messages: - logging.info( - 'Got messages during poking: %s', str(messages)) + self.log.info('Got messages during poking: %s', str(messages)) return messages else: return False From 4ccf498a4abc0e7ed494a340832090b478b4eea6 Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:46 -0400 Subject: [PATCH 051/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 7108f84207bcd..bb27e2f5adc73 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -67,9 +67,7 @@ def poke(self, context): :param context: :return: """ - logging.info( - 'Poking topic: %s, using hook: %s', - str(self.topic), str(self.hook)) + self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) messages = self.hook.get_messages() From e02d4cc4e396f33b3ee77fe07b22d0acc78cd57e Mon Sep 17 00:00:00 2001 From: dan Date: Sat, 23 May 2020 14:46:59 -0400 Subject: [PATCH 052/162] Removed unused imports --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 1 - airflow/providers/apache/kafka/sensors/kafka_sensor.py | 2 -- 2 files changed, 3 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 9919505b4a24d..f6d9fd2428ec0 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -17,7 +17,6 @@ # under the License. from kafka import KafkaProducer -from kafka.producer.future import FutureRecordMetadata from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index bb27e2f5adc73..842643c60ba40 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -16,8 +16,6 @@ # specific language governing permissions and limitations # under the License. -import logging - from cached_property import cached_property from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook From b79063f610a6356d95fb0e37c845d5cdf4fbe667 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 14:18:52 -0400 Subject: [PATCH 053/162] adding more weird local files --- scripts/ci/docker-compose/local.yml | 1 - 1 file changed, 1 deletion(-) mode change 100644 => 100755 scripts/ci/docker-compose/local.yml diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml old mode 100644 new mode 100755 index 619156f5be775..03a4c1a2d3d60 --- a/scripts/ci/docker-compose/local.yml +++ b/scripts/ci/docker-compose/local.yml @@ -14,7 +14,6 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. ---- version: "2.2" services: airflow: From 1e3ced947d1604110b9ff6a678631c5d9cbf7f08 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 14:21:35 -0400 Subject: [PATCH 054/162] Modifying readme to trigger builds --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 647b9f9ac4599..c2869acb4661f 100644 --- a/README.md +++ b/README.md @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. --> - # Apache Airflow [![PyPI version](https://badge.fury.io/py/apache-airflow.svg)](https://badge.fury.io/py/apache-airflow) From 4bb1da6bedc95b0d6d6139ce668c3844d75c06f4 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 14:30:56 -0400 Subject: [PATCH 055/162] Revert "Resolved conflicts in requirements files" This reverts commit 9b213d23ec86d4875dddc9b2dbc7c5a0c5f1f93b. --- requirements/requirements-python3.6.txt | 5 +++-- requirements/requirements-python3.7.txt | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index 562c98c295111..e73a19e92444f 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -56,7 +56,7 @@ azure-batch==9.0.0 azure-common==1.1.25 azure-cosmos==3.2.0 azure-datalake-store==0.0.48 -azure-kusto-data==0.1.0 +azure-kusto-data==0.0.45 azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 @@ -163,7 +163,7 @@ google-cloud-spanner==1.17.1 google-cloud-speech==1.3.2 google-cloud-storage==1.29.0 google-cloud-tasks==1.5.0 -google-cloud-texttospeech==2.0.0 +google-cloud-texttospeech==1.0.1 google-cloud-translate==2.0.1 google-cloud-videointelligence==1.15.0 google-cloud-vision==1.0.0 @@ -227,6 +227,7 @@ moto==1.3.14 msgpack==1.0.0 msrest==0.6.17 msrestazure==0.6.4 +msrest==0.6.14 multi-key-dict==2.0.3 multidict==4.7.6 mypy-extensions==0.4.3 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index 8d6e7542a4821..ae515b109b793 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -56,7 +56,7 @@ azure-batch==9.0.0 azure-common==1.1.25 azure-cosmos==3.2.0 azure-datalake-store==0.0.48 -azure-kusto-data==0.1.0 +azure-kusto-data==0.0.45 azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 @@ -162,7 +162,7 @@ google-cloud-spanner==1.17.1 google-cloud-speech==1.3.2 google-cloud-storage==1.29.0 google-cloud-tasks==1.5.0 -google-cloud-texttospeech==2.0.0 +google-cloud-texttospeech==1.0.1 google-cloud-translate==2.0.1 google-cloud-videointelligence==1.15.0 google-cloud-vision==1.0.0 @@ -181,6 +181,7 @@ humanize==2.5.0 hvac==0.10.4 identify==1.4.23 idna==2.10 +ijson==2.6.1 imagesize==1.2.0 importlib-metadata==1.7.0 inflection==0.5.0 From 169779fb621792ce352f75d806f29aa02ce70319 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 14:52:39 -0400 Subject: [PATCH 056/162] Updated readme to trigger build --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index c2869acb4661f..647b9f9ac4599 100644 --- a/README.md +++ b/README.md @@ -16,6 +16,7 @@ specific language governing permissions and limitations under the License. --> + # Apache Airflow [![PyPI version](https://badge.fury.io/py/apache-airflow.svg)](https://badge.fury.io/py/apache-airflow) From 374343f496e8cf40134ae8264ba22b607f049112 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 15:20:54 -0400 Subject: [PATCH 057/162] updating requirements files --- requirements/requirements-python3.6.txt | 1 - requirements/requirements-python3.7.txt | 3 +++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index e73a19e92444f..5cfa91a9f4d45 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -227,7 +227,6 @@ moto==1.3.14 msgpack==1.0.0 msrest==0.6.17 msrestazure==0.6.4 -msrest==0.6.14 multi-key-dict==2.0.3 multidict==4.7.6 mypy-extensions==0.4.3 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index ae515b109b793..0146e2ce6d1d7 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -75,6 +75,9 @@ blinker==1.4 boto3==1.14.20 boto==2.49.0 botocore==1.17.20 +boto3==1.13.23 +boto==2.49.0 +botocore==1.16.23 bowler==0.8.0 cached-property==1.5.1 cachetools==4.1.1 From a87b26e3bc08e2f071df9fc246fd387237043dc7 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 15:52:00 -0400 Subject: [PATCH 058/162] updated requirements --- requirements/requirements-python3.7.txt | 3 --- 1 file changed, 3 deletions(-) diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index 0146e2ce6d1d7..ae515b109b793 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -75,9 +75,6 @@ blinker==1.4 boto3==1.14.20 boto==2.49.0 botocore==1.17.20 -boto3==1.13.23 -boto==2.49.0 -botocore==1.16.23 bowler==0.8.0 cached-property==1.5.1 cachetools==4.1.1 From 53481787a1fbdfe57f267939a200e7ac463fe7ca Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 12 Feb 2020 13:51:03 -0500 Subject: [PATCH 059/162] Add KafkaConsumerHook, KafkaProduerHook and KafkaSensor MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Dan Ferguson dferguson992@gmail.com Co-authored-by: Hanan Shteingart Co-authored-by: YuanfΞi Zhu --- airflow/contrib/hooks/kafka_consumer_hook.py | 69 ++++++++++++++++++++ airflow/contrib/hooks/kafka_producer_hook.py | 56 ++++++++++++++++ airflow/contrib/sensors/kafka_sensor.py | 50 ++++++++++++++ 3 files changed, 175 insertions(+) create mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py create mode 100644 airflow/contrib/hooks/kafka_producer_hook.py create mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py new file mode 100644 index 0000000000000..335c267b7837f --- /dev/null +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -0,0 +1,69 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaConsumer + + +class KafkaConsumerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + # Disable auto commit as the hook will commit right + # after polling. + conf['enable_auto_commit'] = False + + self.server = '{host}:{port}'.format(**locals()) + self.consumer = KafkaConsumer( + self.topic, + bootstrap_servers=self.server, **conf) + + return self.consumer + + def get_messages(self): + """ + Get all the messages haven't been consumed, it doesn't + block by default, then commit the offset. + :return: + A list of messages + """ + consumer = self.get_conn() + + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms=50) + + consumer.commit() + return messages + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.consumer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py new file mode 100644 index 0000000000000..ffa5d4693e9c2 --- /dev/null +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -0,0 +1,56 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaProducer + + +class KafkaProducerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + conf['enable_auto_commit'] = False + self.server = '{host}:{port}'.format(**locals()) + self.producer = KafkaProducer( + bootstrap_servers=self.server, **conf) + + return self.producer + + def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): + producer = self.get_pub() + future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, + timestamp_ms=timestamp_ms) + producer.close() + return future_record_metadata + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.produer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py new file mode 100644 index 0000000000000..738e63c76c730 --- /dev/null +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -0,0 +1,50 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import logging + +from src.kafka_hook import KafkaConsumerHook +from airflow.operators.sensors import BaseSensorOperator +from airflow.utils import apply_defaults + + +class KafkaSensor(BaseSensorOperator): + """ + Consumes the Kafka message with the specific topic + """ + + @apply_defaults + def __init__(self, conn_id, topic, *args, **kwargs): + """ + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. + :param topic: + the subscribed topic + """ + self.topic = topic + self.hook = KafkaConsumerHook(conn_id, topic) + super(KafkaSensor, self).__init__(*args, **kwargs) + + def poke(self, context): + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) + + messages = self.hook.get_messages() + + logging.info( + 'Got messages during poking: %s', messages) + + return messages or False From 2808b6a4f083e60535bd24b5dae83e7a43b64df4 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 18 Feb 2020 09:05:42 -0500 Subject: [PATCH 060/162] Modifying files in accordance with peer review --- airflow/contrib/hooks/kafka_consumer_hook.py | 24 +++++++++++--------- airflow/contrib/hooks/kafka_producer_hook.py | 24 +++++++++++--------- airflow/contrib/sensors/kafka_sensor.py | 16 +++++++++---- 3 files changed, 38 insertions(+), 26 deletions(-) diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py index 335c267b7837f..1143e2ccfb6a7 100644 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -16,8 +16,8 @@ class KafkaConsumerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): super(KafkaConsumerHook, self).__init__(None) @@ -29,21 +29,21 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT # Disable auto commit as the hook will commit right # after polling. conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.consumer = KafkaConsumer( self.topic, bootstrap_servers=self.server, **conf) return self.consumer - def get_messages(self): + def get_messages(self, timeout_ms=50): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. @@ -51,12 +51,14 @@ def get_messages(self): A list of messages """ consumer = self.get_conn() + try: + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms) - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms=50) - - consumer.commit() + consumer.commit() + finally: + consumer.close() return messages def __repr__(self): diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py index ffa5d4693e9c2..9c1ebb158afe5 100644 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -16,11 +16,11 @@ class KafkaProducerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) + super(KafkaProducerHook, self).__init__(None) self.conn = self.get_connection(conn_id) self.server = None self.consumer = None @@ -29,21 +29,23 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.producer = KafkaProducer( bootstrap_servers=self.server, **conf) return self.producer - def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): - producer = self.get_pub() - future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, - timestamp_ms=timestamp_ms) - producer.close() + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + producer = self.get_conn() + try: + future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, + timestamp_ms=timestamp_ms) + finally: + producer.close() return future_record_metadata def __repr__(self): diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py index 738e63c76c730..e51328cdd63a9 100644 --- a/airflow/contrib/sensors/kafka_sensor.py +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -12,9 +12,11 @@ import logging -from src.kafka_hook import KafkaConsumerHook -from airflow.operators.sensors import BaseSensorOperator -from airflow.utils import apply_defaults +from werkzeug.utils import cached_property + +from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class KafkaSensor(BaseSensorOperator): @@ -22,6 +24,8 @@ class KafkaSensor(BaseSensorOperator): Consumes the Kafka message with the specific topic """ + templated_fields = ('topic',) + @apply_defaults def __init__(self, conn_id, topic, *args, **kwargs): """ @@ -34,9 +38,13 @@ def __init__(self, conn_id, topic, *args, **kwargs): the subscribed topic """ self.topic = topic - self.hook = KafkaConsumerHook(conn_id, topic) + self.conn_id = conn_id super(KafkaSensor, self).__init__(*args, **kwargs) + @cached_property + def hook(self): + return KafkaConsumerHook(self.conn_id, self.topic) + def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', From df9aa3034fee107f53a2bc35a5363e9b251dcc2a Mon Sep 17 00:00:00 2001 From: dan Date: Mon, 2 Mar 2020 15:47:10 -0500 Subject: [PATCH 061/162] Moved files to providers package. Externalized connection logic from init function --- airflow/contrib/hooks/kafka_consumer_hook.py | 71 ------------------- airflow/contrib/hooks/kafka_producer_hook.py | 58 --------------- airflow/contrib/sensors/kafka_sensor.py | 58 --------------- airflow/providers/apache/kafka/__init__.py | 1 - .../apache/kafka/example_dags/__init__.py | 1 - .../providers/apache/kafka/hooks/__init__.py | 1 - .../apache/kafka/hooks/kafka_consumer_hook.py | 49 ++++--------- .../apache/kafka/hooks/kafka_producer_hook.py | 67 +++++------------ .../apache/kafka/sensors/__init__.py | 1 - .../apache/kafka/sensors/kafka_sensor.py | 62 +++++++--------- 10 files changed, 58 insertions(+), 311 deletions(-) delete mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py delete mode 100644 airflow/contrib/hooks/kafka_producer_hook.py delete mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py deleted file mode 100644 index 1143e2ccfb6a7..0000000000000 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ /dev/null @@ -1,71 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaConsumer - - -class KafkaConsumerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - # Disable auto commit as the hook will commit right - # after polling. - conf['enable_auto_commit'] = False - - self.server = f"""{host}:{port}""" - self.consumer = KafkaConsumer( - self.topic, - bootstrap_servers=self.server, **conf) - - return self.consumer - - def get_messages(self, timeout_ms=50): - """ - Get all the messages haven't been consumed, it doesn't - block by default, then commit the offset. - :return: - A list of messages - """ - consumer = self.get_conn() - try: - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms) - - consumer.commit() - finally: - consumer.close() - return messages - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.consumer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py deleted file mode 100644 index 9c1ebb158afe5..0000000000000 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaProducer - - -class KafkaProducerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaProducerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - conf['enable_auto_commit'] = False - self.server = f"""{host}:{port}""" - self.producer = KafkaProducer( - bootstrap_servers=self.server, **conf) - - return self.producer - - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - producer = self.get_conn() - try: - future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, - timestamp_ms=timestamp_ms) - finally: - producer.close() - return future_record_metadata - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.produer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py deleted file mode 100644 index e51328cdd63a9..0000000000000 --- a/airflow/contrib/sensors/kafka_sensor.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import logging - -from werkzeug.utils import cached_property - -from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook -from airflow.sensors.base_sensor_operator import BaseSensorOperator -from airflow.utils.decorators import apply_defaults - - -class KafkaSensor(BaseSensorOperator): - """ - Consumes the Kafka message with the specific topic - """ - - templated_fields = ('topic',) - - @apply_defaults - def __init__(self, conn_id, topic, *args, **kwargs): - """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - :param conn_id: - the kafka broker connection whom this sensor - subscripts against. - :param topic: - the subscribed topic - """ - self.topic = topic - self.conn_id = conn_id - super(KafkaSensor, self).__init__(*args, **kwargs) - - @cached_property - def hook(self): - return KafkaConsumerHook(self.conn_id, self.topic) - - def poke(self, context): - logging.info( - 'Poking topic: %s, using hook: %s', - self.topic, self.hook) - - messages = self.hook.get_messages() - - logging.info( - 'Got messages during poking: %s', messages) - - return messages or False diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py index 217e5db960782..13a83393a9124 100644 --- a/airflow/providers/apache/kafka/__init__.py +++ b/airflow/providers/apache/kafka/__init__.py @@ -1,4 +1,3 @@ -# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information diff --git a/airflow/providers/apache/kafka/example_dags/__init__.py b/airflow/providers/apache/kafka/example_dags/__init__.py index 217e5db960782..13a83393a9124 100644 --- a/airflow/providers/apache/kafka/example_dags/__init__.py +++ b/airflow/providers/apache/kafka/example_dags/__init__.py @@ -1,4 +1,3 @@ -# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information diff --git a/airflow/providers/apache/kafka/hooks/__init__.py b/airflow/providers/apache/kafka/hooks/__init__.py index 217e5db960782..13a83393a9124 100644 --- a/airflow/providers/apache/kafka/hooks/__init__.py +++ b/airflow/providers/apache/kafka/hooks/__init__.py @@ -1,4 +1,3 @@ -# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index df2576069e927..cbb2a4c963839 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -1,31 +1,21 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at # -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 # -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from airflow.hooks import base_hook as BaseHook from kafka import KafkaConsumer -from airflow.hooks.base_hook import BaseHook - class KafkaConsumerHook(BaseHook): - """ - KafkaConsumerHook Class. - """ + DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 @@ -40,13 +30,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self) -> KafkaConsumer: - """ - A Kafka Consumer object. - - :return: - A Kafka Consumer object. - """ + def get_conn(self): if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -61,12 +45,10 @@ def get_conn(self) -> KafkaConsumer: ) return self.consumer - def get_messages(self, timeout_ms=5000) -> dict: + def get_messages(self, timeout_ms=5000): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. - - :param timeout_ms: :return: A list of messages """ @@ -80,10 +62,7 @@ def get_messages(self, timeout_ms=5000) -> dict: def __repr__(self): """ - A pretty version of the connection string. - - :return: - A pretty version of the connection string. + Pretty the hook with the connection info """ connected = self.consumer is not None return ' KafkaProducer: - """ - Returns a Kafka Producer - - :return: - A Kafka Producer object. - """ + def get_conn(self): if not self._conn: - _conn = self.get_connection(self.conn_id) - service_options = _conn.extra_dejson - host = _conn.host or self.DEFAULT_HOST - port = _conn.port or self.DEFAULT_PORT + conn = self.get_connection(self.conn_id) + service_options = conn.extra_dejson + host = conn.host or self.DEFAULT_HOST + port = conn.port or self.DEFAULT_PORT self.server = f"""{host}:{port}""" self.consumer = KafkaProducer( @@ -58,16 +42,6 @@ def get_conn(self) -> KafkaProducer: return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - """ - Sends a message on the specified topic and partition. Keyed messages will be sent in order. - - :param topic: - :param value: - :param key: - :param partition: - :param timestamp_ms: - :return: - """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, @@ -78,12 +52,9 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - A pretty version of the connection string. - - :return: - A pretty version of the connection string. + Pretty the hook with the connection info """ - connected = self.producer is not None + connected = self.produer is not None return '' % \ (connected, self.server, self.topic) diff --git a/airflow/providers/apache/kafka/sensors/__init__.py b/airflow/providers/apache/kafka/sensors/__init__.py index 217e5db960782..13a83393a9124 100644 --- a/airflow/providers/apache/kafka/sensors/__init__.py +++ b/airflow/providers/apache/kafka/sensors/__init__.py @@ -1,4 +1,3 @@ -# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 842643c60ba40..f7043a0294706 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -1,26 +1,21 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at # -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 # -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. -from cached_property import cached_property +import logging -from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook -from airflow.sensors.base_sensor_operator import BaseSensorOperator +from cached_property import cached_property from airflow.utils.decorators import apply_defaults +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook class KafkaSensor(BaseSensorOperator): @@ -37,14 +32,13 @@ class KafkaSensor(BaseSensorOperator): @apply_defaults def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs): """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. :param topic: - :param host: - :param port: - :param args: - :param kwargs: + the subscribed topic """ self.topic = topic self.host = host @@ -53,24 +47,18 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): - """ - Returns a Kafka Consumer Hook - """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): - """ - Checks to see if messages exist on this topic/partition. - - :param context: - :return: - """ - self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) messages = self.hook.get_messages() - if messages: - self.log.info('Got messages during poking: %s', str(messages)) + if messages is not {}: + logging.info( + 'Got messages during poking: %s', messages) return messages else: return False From 163b9cfb8521db0275ebf08eb8da3e0da7f57a1e Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:16:08 -0400 Subject: [PATCH 062/162] produer --> producer --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 4bc60e2a610fd..71945c7d9e87b 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -54,7 +54,7 @@ def __repr__(self): """ Pretty the hook with the connection info """ - connected = self.produer is not None + connected = self.producer is not None return '' % \ (connected, self.server, self.topic) From 4a11c4aa6839cb697b74df8781d54aad210bd422 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:36:16 -0400 Subject: [PATCH 063/162] Added pylint mandated doc strings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 2 ++ .../apache/kafka/hooks/kafka_producer_hook.py | 11 +++++++++++ .../providers/apache/kafka/sensors/kafka_sensor.py | 7 ++++++- 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index cbb2a4c963839..19e039995b810 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -15,7 +15,9 @@ class KafkaConsumerHook(BaseHook): + """ + """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 71945c7d9e87b..de31f06419a77 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -15,7 +15,9 @@ class KafkaProducerHook(BaseHook): + """ + """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 @@ -42,6 +44,15 @@ def get_conn(self): return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + """ + + :param topic: + :param value: + :param key: + :param partition: + :param timestamp_ms: + :return: + """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index f7043a0294706..65cd64af71b92 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -47,12 +47,17 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): + """ + + :return: + KafkaConsumerHook + """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', - self.topic, self.hook) + str(self.topic), str(self.hook)) messages = self.hook.get_messages() From 2c162241cc282b565ddd39a4ccde37a5a5492030 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 10:06:05 -0400 Subject: [PATCH 064/162] Fixing more pylint issues --- airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py | 2 +- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 19e039995b810..17ea7f55c79ec 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -16,7 +16,7 @@ class KafkaConsumerHook(BaseHook): """ - + KafkaConsumerHook Class. """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index de31f06419a77..92be6382b1654 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -16,7 +16,7 @@ class KafkaProducerHook(BaseHook): """ - + KafkaProducerHook Class. """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 65cd64af71b92..1c7ef43137f88 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -61,9 +61,9 @@ def poke(self, context): messages = self.hook.get_messages() - if messages is not {}: + if not messages: logging.info( - 'Got messages during poking: %s', messages) + 'Got messages during poking: %s', str(messages)) return messages else: return False From 9c0ca5d5c999d3f06ce96e033037c4eccbf734d2 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 065/162] Tried to fix docstrings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 17 +++++++++++--- .../apache/kafka/hooks/kafka_producer_hook.py | 17 +++++++++++--- .../apache/kafka/sensors/kafka_sensor.py | 22 +++++++++++++------ 3 files changed, 43 insertions(+), 13 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 17ea7f55c79ec..96472934ab63b 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -32,7 +32,13 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self): + def get_conn(self) -> KafkaConsumer: + """ + A Kafka Consumer object. + + :return: + A Kafka Consumer object. + """ if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -47,10 +53,12 @@ def get_conn(self): ) return self.consumer - def get_messages(self, timeout_ms=5000): + def get_messages(self, timeout_ms=5000) -> dict: """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. + + :param timeout_ms: :return: A list of messages """ @@ -64,7 +72,10 @@ def get_messages(self, timeout_ms=5000): def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.consumer is not None return ' KafkaProducer: + """ + Returns a Kafka Producer + + :return: + A Kafka Producer object. + """ if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -43,8 +50,9 @@ def get_conn(self): ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: """ + Sends a message on the specified topic and partition. Keyed messages will be sent in order. :param topic: :param value: @@ -63,7 +71,10 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.producer is not None return ' Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 066/162] Tried to fix docstrings --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 4056990ae6fc0..47cd19a34e6b2 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -50,7 +50,7 @@ def get_conn(self) -> KafkaProducer: ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): """ Sends a message on the specified topic and partition. Keyed messages will be sent in order. From 646bcf0157a51e6da2e725ff1342217b0640057c Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 9 May 2020 23:41:58 +0100 Subject: [PATCH 067/162] Apply suggestions from code review --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 1a82ac5618ead..3a8f2bde8d210 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -49,10 +49,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): """ - Returns a Kafka Consumer Hook - - :return: - KafkaConsumerHook + Returns a Kafka Consumer Hook """ return KafkaConsumerHook(self.topic, self.host, self.port) From 0193e675477b6204395a100be95e875453c7426b Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:28 -0400 Subject: [PATCH 068/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 3a8f2bde8d210..232d40eb19b7d 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -67,8 +67,7 @@ def poke(self, context): messages = self.hook.get_messages() if messages: - logging.info( - 'Got messages during poking: %s', str(messages)) + self.log.info('Got messages during poking: %s', str(messages)) return messages else: return False From c092186e83956d864c9a9cbc65548262eb404dfa Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:46 -0400 Subject: [PATCH 069/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 232d40eb19b7d..61f0c6ad784e1 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -60,9 +60,7 @@ def poke(self, context): :param context: :return: """ - logging.info( - 'Poking topic: %s, using hook: %s', - str(self.topic), str(self.hook)) + self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) messages = self.hook.get_messages() From 62b4e5ad081d559d101bce65284ea235ba504631 Mon Sep 17 00:00:00 2001 From: dan Date: Sat, 23 May 2020 14:46:59 -0400 Subject: [PATCH 070/162] Removed unused imports --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 1 - airflow/providers/apache/kafka/sensors/kafka_sensor.py | 2 -- 2 files changed, 3 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 47cd19a34e6b2..6d8391a87eb3b 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -12,7 +12,6 @@ from airflow.hooks import base_hook as BaseHook from kafka import KafkaProducer -from kafka.producer.future import FutureRecordMetadata class KafkaProducerHook(BaseHook): diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 61f0c6ad784e1..46503c838d1b5 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -10,8 +10,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -import logging - from cached_property import cached_property from airflow.utils.decorators import apply_defaults from airflow.sensors.base_sensor_operator import BaseSensorOperator From c8f2995399a5564af6fcc0e76e070a0d2ff228f8 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 18:24:31 -0400 Subject: [PATCH 071/162] updating requirements again --- requirements/requirements-python3.6.txt | 3 +++ requirements/requirements-python3.7.txt | 5 ++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index 5cfa91a9f4d45..c2bd5fa5535cd 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -213,6 +213,7 @@ kubernetes==11.0.0 kylinpy==2.8.1 lazy-object-proxy==1.5.0 ldap3==2.7 +libcst==0.3.6 lockfile==0.12.2 marshmallow-enum==1.5.1 marshmallow-oneofschema==2.0.1 @@ -266,6 +267,7 @@ presto-python-client==0.7.0 prison==0.1.3 prometheus-client==0.8.0 prompt-toolkit==3.0.5 +proto-plus==0.4.0 protobuf==3.12.2 psutil==5.7.0 psycopg2-binary==2.8.5 @@ -379,6 +381,7 @@ traitlets==4.3.3 typed-ast==1.4.1 typing-extensions==3.7.4.2 typing==3.7.4.3 +typing-inspect==0.6.0 tzlocal==1.5.1 unicodecsv==0.14.1 uritemplate==3.0.1 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index ae515b109b793..ced25f19a306a 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -210,6 +210,7 @@ kubernetes==11.0.0 kylinpy==2.8.1 lazy-object-proxy==1.5.0 ldap3==2.7 +libcst==0.3.6 lockfile==0.12.2 marshmallow-enum==1.5.1 marshmallow-oneofschema==2.0.1 @@ -262,6 +263,7 @@ presto-python-client==0.7.0 prison==0.1.3 prometheus-client==0.8.0 prompt-toolkit==3.0.5 +proto-plus==0.4.0 protobuf==3.12.2 psutil==5.7.0 psycopg2-binary==2.8.5 @@ -373,6 +375,7 @@ tqdm==4.47.0 traitlets==4.3.3 typed-ast==1.4.1 typing-extensions==3.7.4.2 +typing-inspect==0.6.0 tzlocal==1.5.1 unicodecsv==0.14.1 uritemplate==3.0.1 @@ -383,7 +386,7 @@ virtualenv==20.0.26 watchtower==0.7.3 wcwidth==0.2.5 websocket-client==0.57.0 -wrapt==1.12.1 +wrapt==1.11.2 xmltodict==0.12.0 yamllint==1.23.0 yandexcloud==0.43.0 From 857f44edaf164c83dd2b21abc1dc807fd70d70a6 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 21:04:17 -0400 Subject: [PATCH 072/162] Fixed yaml linting issues --- scripts/ci/docker-compose/local.yml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml index 03a4c1a2d3d60..cda6ce89aaa84 100755 --- a/scripts/ci/docker-compose/local.yml +++ b/scripts/ci/docker-compose/local.yml @@ -57,5 +57,15 @@ services: - ../../../tmp:/tmp:cached - ../../../metastore_browser:/opt/airflow/metastore_browser:cached # END automatically generated volumes from LOCAL_MOUNTS in _local_mounts.sh + - ../../../tmp:/opt/airflow/tmp:cached + # END automatically generated volumes from LOCAL_MOUNTS in + # _local_mounts.sh + environment: + - HOST_USER_ID + - HOST_GROUP_ID + - HOST_HOME=${HOME} + - HOST_AIRFLOW_SOURCES=${AIRFLOW_SOURCES} + - HOST_OS + - PYTHONDONTWRITEBYTECODE ports: - "${WEBSERVER_HOST_PORT}:8080" From 5974f63a50f9e0b6b84c2d3de8b68c07d499c1da Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 12 Feb 2020 13:51:03 -0500 Subject: [PATCH 073/162] Add KafkaConsumerHook, KafkaProduerHook and KafkaSensor MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Dan Ferguson dferguson992@gmail.com Co-authored-by: Hanan Shteingart Co-authored-by: YuanfΞi Zhu --- airflow/contrib/hooks/kafka_consumer_hook.py | 69 ++++++++++++++++++++ airflow/contrib/hooks/kafka_producer_hook.py | 56 ++++++++++++++++ airflow/contrib/sensors/kafka_sensor.py | 50 ++++++++++++++ 3 files changed, 175 insertions(+) create mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py create mode 100644 airflow/contrib/hooks/kafka_producer_hook.py create mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py new file mode 100644 index 0000000000000..335c267b7837f --- /dev/null +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -0,0 +1,69 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaConsumer + + +class KafkaConsumerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + # Disable auto commit as the hook will commit right + # after polling. + conf['enable_auto_commit'] = False + + self.server = '{host}:{port}'.format(**locals()) + self.consumer = KafkaConsumer( + self.topic, + bootstrap_servers=self.server, **conf) + + return self.consumer + + def get_messages(self): + """ + Get all the messages haven't been consumed, it doesn't + block by default, then commit the offset. + :return: + A list of messages + """ + consumer = self.get_conn() + + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms=50) + + consumer.commit() + return messages + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.consumer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py new file mode 100644 index 0000000000000..ffa5d4693e9c2 --- /dev/null +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -0,0 +1,56 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaProducer + + +class KafkaProducerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + conf['enable_auto_commit'] = False + self.server = '{host}:{port}'.format(**locals()) + self.producer = KafkaProducer( + bootstrap_servers=self.server, **conf) + + return self.producer + + def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): + producer = self.get_pub() + future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, + timestamp_ms=timestamp_ms) + producer.close() + return future_record_metadata + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.produer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py new file mode 100644 index 0000000000000..738e63c76c730 --- /dev/null +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -0,0 +1,50 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import logging + +from src.kafka_hook import KafkaConsumerHook +from airflow.operators.sensors import BaseSensorOperator +from airflow.utils import apply_defaults + + +class KafkaSensor(BaseSensorOperator): + """ + Consumes the Kafka message with the specific topic + """ + + @apply_defaults + def __init__(self, conn_id, topic, *args, **kwargs): + """ + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. + :param topic: + the subscribed topic + """ + self.topic = topic + self.hook = KafkaConsumerHook(conn_id, topic) + super(KafkaSensor, self).__init__(*args, **kwargs) + + def poke(self, context): + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) + + messages = self.hook.get_messages() + + logging.info( + 'Got messages during poking: %s', messages) + + return messages or False From d4cbb299d62807235810f1ca097a632039f2e629 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 18 Feb 2020 09:05:42 -0500 Subject: [PATCH 074/162] Modifying files in accordance with peer review --- airflow/contrib/hooks/kafka_consumer_hook.py | 24 +++++++++++--------- airflow/contrib/hooks/kafka_producer_hook.py | 24 +++++++++++--------- airflow/contrib/sensors/kafka_sensor.py | 16 +++++++++---- 3 files changed, 38 insertions(+), 26 deletions(-) diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py index 335c267b7837f..1143e2ccfb6a7 100644 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -16,8 +16,8 @@ class KafkaConsumerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): super(KafkaConsumerHook, self).__init__(None) @@ -29,21 +29,21 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT # Disable auto commit as the hook will commit right # after polling. conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.consumer = KafkaConsumer( self.topic, bootstrap_servers=self.server, **conf) return self.consumer - def get_messages(self): + def get_messages(self, timeout_ms=50): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. @@ -51,12 +51,14 @@ def get_messages(self): A list of messages """ consumer = self.get_conn() + try: + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms) - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms=50) - - consumer.commit() + consumer.commit() + finally: + consumer.close() return messages def __repr__(self): diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py index ffa5d4693e9c2..9c1ebb158afe5 100644 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -16,11 +16,11 @@ class KafkaProducerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) + super(KafkaProducerHook, self).__init__(None) self.conn = self.get_connection(conn_id) self.server = None self.consumer = None @@ -29,21 +29,23 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.producer = KafkaProducer( bootstrap_servers=self.server, **conf) return self.producer - def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): - producer = self.get_pub() - future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, - timestamp_ms=timestamp_ms) - producer.close() + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + producer = self.get_conn() + try: + future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, + timestamp_ms=timestamp_ms) + finally: + producer.close() return future_record_metadata def __repr__(self): diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py index 738e63c76c730..e51328cdd63a9 100644 --- a/airflow/contrib/sensors/kafka_sensor.py +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -12,9 +12,11 @@ import logging -from src.kafka_hook import KafkaConsumerHook -from airflow.operators.sensors import BaseSensorOperator -from airflow.utils import apply_defaults +from werkzeug.utils import cached_property + +from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class KafkaSensor(BaseSensorOperator): @@ -22,6 +24,8 @@ class KafkaSensor(BaseSensorOperator): Consumes the Kafka message with the specific topic """ + templated_fields = ('topic',) + @apply_defaults def __init__(self, conn_id, topic, *args, **kwargs): """ @@ -34,9 +38,13 @@ def __init__(self, conn_id, topic, *args, **kwargs): the subscribed topic """ self.topic = topic - self.hook = KafkaConsumerHook(conn_id, topic) + self.conn_id = conn_id super(KafkaSensor, self).__init__(*args, **kwargs) + @cached_property + def hook(self): + return KafkaConsumerHook(self.conn_id, self.topic) + def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', From bae395f19ea206d9c87d50fb7278f00c016c0e8a Mon Sep 17 00:00:00 2001 From: dan Date: Mon, 2 Mar 2020 15:47:10 -0500 Subject: [PATCH 075/162] Moved files to providers package. Externalized connection logic from init function --- airflow/contrib/hooks/kafka_consumer_hook.py | 71 ------------------- airflow/contrib/hooks/kafka_producer_hook.py | 58 --------------- airflow/contrib/sensors/kafka_sensor.py | 58 --------------- .../kafka/example_dags/example_kafka_dag.py | 28 ++++++++ .../apache/kafka/hooks/kafka_consumer_hook.py | 21 ++---- .../apache/kafka/hooks/kafka_producer_hook.py | 29 ++------ .../apache/kafka/sensors/kafka_sensor.py | 33 ++++----- 7 files changed, 50 insertions(+), 248 deletions(-) delete mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py delete mode 100644 airflow/contrib/hooks/kafka_producer_hook.py delete mode 100644 airflow/contrib/sensors/kafka_sensor.py create mode 100644 airflow/providers/apache/kafka/example_dags/example_kafka_dag.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py deleted file mode 100644 index 1143e2ccfb6a7..0000000000000 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ /dev/null @@ -1,71 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaConsumer - - -class KafkaConsumerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - # Disable auto commit as the hook will commit right - # after polling. - conf['enable_auto_commit'] = False - - self.server = f"""{host}:{port}""" - self.consumer = KafkaConsumer( - self.topic, - bootstrap_servers=self.server, **conf) - - return self.consumer - - def get_messages(self, timeout_ms=50): - """ - Get all the messages haven't been consumed, it doesn't - block by default, then commit the offset. - :return: - A list of messages - """ - consumer = self.get_conn() - try: - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms) - - consumer.commit() - finally: - consumer.close() - return messages - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.consumer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py deleted file mode 100644 index 9c1ebb158afe5..0000000000000 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaProducer - - -class KafkaProducerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaProducerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - conf['enable_auto_commit'] = False - self.server = f"""{host}:{port}""" - self.producer = KafkaProducer( - bootstrap_servers=self.server, **conf) - - return self.producer - - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - producer = self.get_conn() - try: - future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, - timestamp_ms=timestamp_ms) - finally: - producer.close() - return future_record_metadata - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.produer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py deleted file mode 100644 index e51328cdd63a9..0000000000000 --- a/airflow/contrib/sensors/kafka_sensor.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import logging - -from werkzeug.utils import cached_property - -from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook -from airflow.sensors.base_sensor_operator import BaseSensorOperator -from airflow.utils.decorators import apply_defaults - - -class KafkaSensor(BaseSensorOperator): - """ - Consumes the Kafka message with the specific topic - """ - - templated_fields = ('topic',) - - @apply_defaults - def __init__(self, conn_id, topic, *args, **kwargs): - """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - :param conn_id: - the kafka broker connection whom this sensor - subscripts against. - :param topic: - the subscribed topic - """ - self.topic = topic - self.conn_id = conn_id - super(KafkaSensor, self).__init__(*args, **kwargs) - - @cached_property - def hook(self): - return KafkaConsumerHook(self.conn_id, self.topic) - - def poke(self, context): - logging.info( - 'Poking topic: %s, using hook: %s', - self.topic, self.hook) - - messages = self.hook.get_messages() - - logging.info( - 'Got messages during poking: %s', messages) - - return messages or False diff --git a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py new file mode 100644 index 0000000000000..6b053d007ac56 --- /dev/null +++ b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py @@ -0,0 +1,28 @@ + +from airflow import DAG +from datetime import timedelta + +from airflow.utils.timezone import datetime +from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor + +DAG_ID = "example_kafka_dag" +dag_start_date = datetime(2015, 6, 1, hour=20, tzinfo=None) +default_args = { + 'owner': '@Ferg_In', + 'depends_on_past': False, + 'start_date': dag_start_date, + 'email': ['dferguson992@gmail.com'], + 'provide_context': True, + 'retries': 3, + 'retry_delay': timedelta(minutes=5), +} + +with DAG(dag_id=DAG_ID, default_args=default_args, schedule_interval=None, + max_active_runs=1, concurrency=4, catchup=False) as dag: + + sensor = KafkaSensor( + task_id='trigger', + topic='', + host='', + port='' + ) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 96472934ab63b..cbb2a4c963839 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -15,9 +15,7 @@ class KafkaConsumerHook(BaseHook): - """ - KafkaConsumerHook Class. - """ + DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 @@ -32,13 +30,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self) -> KafkaConsumer: - """ - A Kafka Consumer object. - - :return: - A Kafka Consumer object. - """ + def get_conn(self): if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -53,12 +45,10 @@ def get_conn(self) -> KafkaConsumer: ) return self.consumer - def get_messages(self, timeout_ms=5000) -> dict: + def get_messages(self, timeout_ms=5000): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. - - :param timeout_ms: :return: A list of messages """ @@ -72,10 +62,7 @@ def get_messages(self, timeout_ms=5000) -> dict: def __repr__(self): """ - A pretty version of the connection string. - - :return: - A pretty version of the connection string. + Pretty the hook with the connection info """ connected = self.consumer is not None return ' KafkaProducer: - """ - Returns a Kafka Producer - - :return: - A Kafka Producer object. - """ + def get_conn(self): if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -50,16 +42,6 @@ def get_conn(self) -> KafkaProducer: return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - """ - Sends a message on the specified topic and partition. Keyed messages will be sent in order. - - :param topic: - :param value: - :param key: - :param partition: - :param timestamp_ms: - :return: - """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, @@ -70,12 +52,9 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - A pretty version of the connection string. - - :return: - A pretty version of the connection string. + Pretty the hook with the connection info """ - connected = self.producer is not None + connected = self.produer is not None return '' % \ (connected, self.server, self.topic) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 46503c838d1b5..f7043a0294706 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -10,6 +10,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +import logging + from cached_property import cached_property from airflow.utils.decorators import apply_defaults from airflow.sensors.base_sensor_operator import BaseSensorOperator @@ -30,14 +32,13 @@ class KafkaSensor(BaseSensorOperator): @apply_defaults def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs): """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. :param topic: - :param host: - :param port: - :param args: - :param kwargs: + the subscribed topic """ self.topic = topic self.host = host @@ -46,24 +47,18 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): - """ - Returns a Kafka Consumer Hook - """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): - """ - Checks to see if messages exist on this topic/partition. - - :param context: - :return: - """ - self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) messages = self.hook.get_messages() - if messages: - self.log.info('Got messages during poking: %s', str(messages)) + if messages is not {}: + logging.info( + 'Got messages during poking: %s', messages) return messages else: return False From 7976053ac36cef85e0c5af93600581499f415efb Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 11 Mar 2020 09:06:55 -0400 Subject: [PATCH 076/162] Fixing Travis CI Linting errors --- airflow/providers/apache/kafka/__init__.py | 1 + .../providers/apache/kafka/hooks/__init__.py | 1 + .../apache/kafka/hooks/kafka_consumer_hook.py | 2 +- .../apache/kafka/hooks/kafka_producer_hook.py | 2 +- .../apache/kafka/sensors/kafka_sensor.py | 24 ++++++++++++------- 5 files changed, 19 insertions(+), 11 deletions(-) diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py index 13a83393a9124..217e5db960782 100644 --- a/airflow/providers/apache/kafka/__init__.py +++ b/airflow/providers/apache/kafka/__init__.py @@ -1,3 +1,4 @@ +# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information diff --git a/airflow/providers/apache/kafka/hooks/__init__.py b/airflow/providers/apache/kafka/hooks/__init__.py index 13a83393a9124..217e5db960782 100644 --- a/airflow/providers/apache/kafka/hooks/__init__.py +++ b/airflow/providers/apache/kafka/hooks/__init__.py @@ -1,3 +1,4 @@ +# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index cbb2a4c963839..968514398a20b 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -10,8 +10,8 @@ # See the License for the specific language governing permissions and # limitations under the License. -from airflow.hooks import base_hook as BaseHook from kafka import KafkaConsumer +from airflow.hooks.base_hook import BaseHook class KafkaConsumerHook(BaseHook): diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 4bc60e2a610fd..7d74110012529 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -10,8 +10,8 @@ # See the License for the specific language governing permissions and # limitations under the License. -from airflow.hooks import base_hook as BaseHook from kafka import KafkaProducer +from airflow.hooks.base_hook import BaseHook class KafkaProducerHook(BaseHook): diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index f7043a0294706..c5096d8734fcd 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -1,14 +1,20 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. import logging From 212491d2aa2fb4cd062b71f0e3e40e62a445908d Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 11 Mar 2020 09:06:55 -0400 Subject: [PATCH 077/162] Fixing Travis CI Linting errors --- .../apache/kafka/example_dags/__init__.py | 1 + .../kafka/example_dags/example_kafka_dag.py | 17 +++++++++ .../apache/kafka/hooks/kafka_consumer_hook.py | 25 ++++++++----- .../apache/kafka/hooks/kafka_producer_hook.py | 35 +++++++++++-------- .../apache/kafka/sensors/__init__.py | 1 + 5 files changed, 56 insertions(+), 23 deletions(-) diff --git a/airflow/providers/apache/kafka/example_dags/__init__.py b/airflow/providers/apache/kafka/example_dags/__init__.py index 13a83393a9124..217e5db960782 100644 --- a/airflow/providers/apache/kafka/example_dags/__init__.py +++ b/airflow/providers/apache/kafka/example_dags/__init__.py @@ -1,3 +1,4 @@ +# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information diff --git a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py index 6b053d007ac56..eb89df6b7cca8 100644 --- a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py +++ b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. from airflow import DAG from datetime import timedelta diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 968514398a20b..bb4895fab25fb 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -1,14 +1,21 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + from kafka import KafkaConsumer from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 7d74110012529..e7f70f2ef1aec 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -1,14 +1,20 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. from kafka import KafkaProducer from airflow.hooks.base_hook import BaseHook @@ -21,7 +27,8 @@ class KafkaProducerHook(BaseHook): def __init__(self, conn_id, topic): super(KafkaProducerHook, self).__init__(None) - self.conn = None + self.conn_id = conn_id + self._conn = None self.server = None self.consumer = None self.producer = None @@ -29,10 +36,10 @@ def __init__(self, conn_id, topic): def get_conn(self): if not self._conn: - conn = self.get_connection(self.conn_id) - service_options = conn.extra_dejson - host = conn.host or self.DEFAULT_HOST - port = conn.port or self.DEFAULT_PORT + _conn = self.get_connection(self.conn_id) + service_options = _conn.extra_dejson + host = _conn.host or self.DEFAULT_HOST + port = _conn.port or self.DEFAULT_PORT self.server = f"""{host}:{port}""" self.consumer = KafkaProducer( diff --git a/airflow/providers/apache/kafka/sensors/__init__.py b/airflow/providers/apache/kafka/sensors/__init__.py index 13a83393a9124..217e5db960782 100644 --- a/airflow/providers/apache/kafka/sensors/__init__.py +++ b/airflow/providers/apache/kafka/sensors/__init__.py @@ -1,3 +1,4 @@ +# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information From 53b5f97ce4bd13f9041fdede8e603ae333b1d954 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:16:08 -0400 Subject: [PATCH 078/162] produer --> producer --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index e7f70f2ef1aec..0d5f487c3ff85 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -61,7 +61,7 @@ def __repr__(self): """ Pretty the hook with the connection info """ - connected = self.produer is not None + connected = self.producer is not None return '' % \ (connected, self.server, self.topic) From 6813f27f5ed2626a902a683fab155657fbe9c385 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:36:16 -0400 Subject: [PATCH 079/162] Added pylint mandated doc strings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 2 ++ .../apache/kafka/hooks/kafka_producer_hook.py | 11 +++++++++++ .../providers/apache/kafka/sensors/kafka_sensor.py | 7 ++++++- 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index bb4895fab25fb..a9abdcef3c148 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -22,7 +22,9 @@ class KafkaConsumerHook(BaseHook): + """ + """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 0d5f487c3ff85..3faf5c379d33e 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -21,7 +21,9 @@ class KafkaProducerHook(BaseHook): + """ + """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 @@ -49,6 +51,15 @@ def get_conn(self): return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + """ + + :param topic: + :param value: + :param key: + :param partition: + :param timestamp_ms: + :return: + """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index c5096d8734fcd..e8119da6fc1e1 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -53,12 +53,17 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): + """ + + :return: + KafkaConsumerHook + """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', - self.topic, self.hook) + str(self.topic), str(self.hook)) messages = self.hook.get_messages() From 15e81a0f2dabfd7377288b7a327cbe1a618e735c Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 10:06:05 -0400 Subject: [PATCH 080/162] Fixing more pylint issues --- airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py | 2 +- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index a9abdcef3c148..9270c6bbf25c5 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -23,7 +23,7 @@ class KafkaConsumerHook(BaseHook): """ - + KafkaConsumerHook Class. """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 3faf5c379d33e..845686fab9979 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -22,7 +22,7 @@ class KafkaProducerHook(BaseHook): """ - + KafkaProducerHook Class. """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index e8119da6fc1e1..48ab157e94078 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -67,9 +67,9 @@ def poke(self, context): messages = self.hook.get_messages() - if messages is not {}: + if not messages: logging.info( - 'Got messages during poking: %s', messages) + 'Got messages during poking: %s', str(messages)) return messages else: return False From 29d333d6a90a84b193ac2238d1bc90da18ce2bdd Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 081/162] Tried to fix docstrings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 17 +++++++++++--- .../apache/kafka/hooks/kafka_producer_hook.py | 17 +++++++++++--- .../apache/kafka/sensors/kafka_sensor.py | 22 +++++++++++++------ 3 files changed, 43 insertions(+), 13 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 9270c6bbf25c5..6b58bd3091d1a 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -39,7 +39,13 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self): + def get_conn(self) -> KafkaConsumer: + """ + A Kafka Consumer object. + + :return: + A Kafka Consumer object. + """ if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -54,10 +60,12 @@ def get_conn(self): ) return self.consumer - def get_messages(self, timeout_ms=5000): + def get_messages(self, timeout_ms=5000) -> dict: """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. + + :param timeout_ms: :return: A list of messages """ @@ -71,7 +79,10 @@ def get_messages(self, timeout_ms=5000): def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.consumer is not None return ' KafkaProducer: + """ + Returns a Kafka Producer + + :return: + A Kafka Producer object. + """ if not self._conn: _conn = self.get_connection(self.conn_id) service_options = _conn.extra_dejson @@ -50,8 +57,9 @@ def get_conn(self): ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: """ + Sends a message on the specified topic and partition. Keyed messages will be sent in order. :param topic: :param value: @@ -70,7 +78,10 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.producer is not None return ' Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 082/162] Tried to fix docstrings --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index ce50246e7f000..d67ce1eee97e7 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -57,7 +57,7 @@ def get_conn(self) -> KafkaProducer: ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): """ Sends a message on the specified topic and partition. Keyed messages will be sent in order. From 28960e8fe06b3f91fc7f62e0c0043bf03e6e04df Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 9 May 2020 23:41:58 +0100 Subject: [PATCH 083/162] Apply suggestions from code review --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 158eae23270d8..636a29035a5d3 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -55,10 +55,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): """ - Returns a Kafka Consumer Hook - - :return: - KafkaConsumerHook + Returns a Kafka Consumer Hook """ return KafkaConsumerHook(self.topic, self.host, self.port) From 605195e6de9e45b1eb697d2a79f85b915b37ee3a Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:28 -0400 Subject: [PATCH 084/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 636a29035a5d3..58d3e203ccc44 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -73,8 +73,7 @@ def poke(self, context): messages = self.hook.get_messages() if messages: - logging.info( - 'Got messages during poking: %s', str(messages)) + self.log.info('Got messages during poking: %s', str(messages)) return messages else: return False From a6a2f74f5a735c8834d6431031885dfd108f799a Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:46 -0400 Subject: [PATCH 085/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 58d3e203ccc44..16c1c5ed4405d 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -66,9 +66,7 @@ def poke(self, context): :param context: :return: """ - logging.info( - 'Poking topic: %s, using hook: %s', - str(self.topic), str(self.hook)) + self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) messages = self.hook.get_messages() From 556425a5d4cd18eed4c840bc0e524fe001b20fde Mon Sep 17 00:00:00 2001 From: dan Date: Sat, 23 May 2020 14:46:59 -0400 Subject: [PATCH 086/162] Removed unused imports --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 16c1c5ed4405d..481c69a321557 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -16,8 +16,6 @@ # specific language governing permissions and limitations # under the License. -import logging - from cached_property import cached_property from airflow.utils.decorators import apply_defaults from airflow.sensors.base_sensor_operator import BaseSensorOperator From 3f096dca43bf5b6265aa3a20b46e796b2dd10db3 Mon Sep 17 00:00:00 2001 From: dan Date: Sat, 23 May 2020 14:46:59 -0400 Subject: [PATCH 087/162] Removed unused imports --- setup.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/setup.py b/setup.py index a294c7a0a7ff4..eaf744725f366 100644 --- a/setup.py +++ b/setup.py @@ -500,8 +500,8 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version "apache.druid": druid, "apache.hdfs": hdfs, "apache.hive": hive, - "apache.kylin": kylin, "apache.kafka": kafka, + "apache.kylin": kylin, "apache.livy": [], "apache.pig": [], "apache.pinot": pinot, @@ -565,7 +565,10 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version "apache.druid": druid, "apache.hdfs": hdfs, "apache.hive": hive, +<<<<<<< HEAD "apache.kylin": kylin, +======= +>>>>>>> 9ad001e9a... Removed unused imports "apache.kafka": kafka, "apache.pinot": pinot, "apache.webhdfs": webhdfs, From 48b7f9fb2438573e18489b3699718e4562f5553c Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 12:05:40 -0400 Subject: [PATCH 088/162] Updated the requirements files to contain the kafka dependencies --- requirements/requirements-python3.6.txt | 4 ++-- requirements/requirements-python3.7.txt | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index c2bd5fa5535cd..fed4f599bec4b 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -56,7 +56,7 @@ azure-batch==9.0.0 azure-common==1.1.25 azure-cosmos==3.2.0 azure-datalake-store==0.0.48 -azure-kusto-data==0.0.45 +azure-kusto-data==0.1.0 azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 @@ -163,7 +163,7 @@ google-cloud-spanner==1.17.1 google-cloud-speech==1.3.2 google-cloud-storage==1.29.0 google-cloud-tasks==1.5.0 -google-cloud-texttospeech==1.0.1 +google-cloud-texttospeech==2.0.0 google-cloud-translate==2.0.1 google-cloud-videointelligence==1.15.0 google-cloud-vision==1.0.0 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index ced25f19a306a..6298069a2faf2 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -56,7 +56,7 @@ azure-batch==9.0.0 azure-common==1.1.25 azure-cosmos==3.2.0 azure-datalake-store==0.0.48 -azure-kusto-data==0.0.45 +azure-kusto-data==0.1.0 azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 @@ -162,7 +162,7 @@ google-cloud-spanner==1.17.1 google-cloud-speech==1.3.2 google-cloud-storage==1.29.0 google-cloud-tasks==1.5.0 -google-cloud-texttospeech==1.0.1 +google-cloud-texttospeech==2.0.0 google-cloud-translate==2.0.1 google-cloud-videointelligence==1.15.0 google-cloud-vision==1.0.0 From de2ad605f97141d25ba226beeebd6931b0ec38f9 Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 12 Feb 2020 13:51:03 -0500 Subject: [PATCH 089/162] Add KafkaConsumerHook, KafkaProduerHook and KafkaSensor MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Dan Ferguson dferguson992@gmail.com Co-authored-by: Hanan Shteingart Co-authored-by: YuanfΞi Zhu --- airflow/contrib/hooks/kafka_consumer_hook.py | 69 ++++++++++++++++++++ airflow/contrib/hooks/kafka_producer_hook.py | 56 ++++++++++++++++ airflow/contrib/sensors/kafka_sensor.py | 50 ++++++++++++++ 3 files changed, 175 insertions(+) create mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py create mode 100644 airflow/contrib/hooks/kafka_producer_hook.py create mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py new file mode 100644 index 0000000000000..335c267b7837f --- /dev/null +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -0,0 +1,69 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaConsumer + + +class KafkaConsumerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + # Disable auto commit as the hook will commit right + # after polling. + conf['enable_auto_commit'] = False + + self.server = '{host}:{port}'.format(**locals()) + self.consumer = KafkaConsumer( + self.topic, + bootstrap_servers=self.server, **conf) + + return self.consumer + + def get_messages(self): + """ + Get all the messages haven't been consumed, it doesn't + block by default, then commit the offset. + :return: + A list of messages + """ + consumer = self.get_conn() + + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms=50) + + consumer.commit() + return messages + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.consumer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py new file mode 100644 index 0000000000000..ffa5d4693e9c2 --- /dev/null +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -0,0 +1,56 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaProducer + + +class KafkaProducerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + conf['enable_auto_commit'] = False + self.server = '{host}:{port}'.format(**locals()) + self.producer = KafkaProducer( + bootstrap_servers=self.server, **conf) + + return self.producer + + def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): + producer = self.get_pub() + future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, + timestamp_ms=timestamp_ms) + producer.close() + return future_record_metadata + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.produer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py new file mode 100644 index 0000000000000..738e63c76c730 --- /dev/null +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -0,0 +1,50 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import logging + +from src.kafka_hook import KafkaConsumerHook +from airflow.operators.sensors import BaseSensorOperator +from airflow.utils import apply_defaults + + +class KafkaSensor(BaseSensorOperator): + """ + Consumes the Kafka message with the specific topic + """ + + @apply_defaults + def __init__(self, conn_id, topic, *args, **kwargs): + """ + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. + :param topic: + the subscribed topic + """ + self.topic = topic + self.hook = KafkaConsumerHook(conn_id, topic) + super(KafkaSensor, self).__init__(*args, **kwargs) + + def poke(self, context): + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) + + messages = self.hook.get_messages() + + logging.info( + 'Got messages during poking: %s', messages) + + return messages or False From 9d9e41fe063fd8f6183c722a762cc7be993907ab Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 18 Feb 2020 09:05:42 -0500 Subject: [PATCH 090/162] Modifying files in accordance with peer review --- airflow/contrib/hooks/kafka_consumer_hook.py | 24 +++++++++++--------- airflow/contrib/hooks/kafka_producer_hook.py | 24 +++++++++++--------- airflow/contrib/sensors/kafka_sensor.py | 16 +++++++++---- 3 files changed, 38 insertions(+), 26 deletions(-) diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py index 335c267b7837f..1143e2ccfb6a7 100644 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -16,8 +16,8 @@ class KafkaConsumerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): super(KafkaConsumerHook, self).__init__(None) @@ -29,21 +29,21 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT # Disable auto commit as the hook will commit right # after polling. conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.consumer = KafkaConsumer( self.topic, bootstrap_servers=self.server, **conf) return self.consumer - def get_messages(self): + def get_messages(self, timeout_ms=50): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. @@ -51,12 +51,14 @@ def get_messages(self): A list of messages """ consumer = self.get_conn() + try: + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms) - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms=50) - - consumer.commit() + consumer.commit() + finally: + consumer.close() return messages def __repr__(self): diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py index ffa5d4693e9c2..9c1ebb158afe5 100644 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -16,11 +16,11 @@ class KafkaProducerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) + super(KafkaProducerHook, self).__init__(None) self.conn = self.get_connection(conn_id) self.server = None self.consumer = None @@ -29,21 +29,23 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.producer = KafkaProducer( bootstrap_servers=self.server, **conf) return self.producer - def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): - producer = self.get_pub() - future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, - timestamp_ms=timestamp_ms) - producer.close() + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + producer = self.get_conn() + try: + future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, + timestamp_ms=timestamp_ms) + finally: + producer.close() return future_record_metadata def __repr__(self): diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py index 738e63c76c730..e51328cdd63a9 100644 --- a/airflow/contrib/sensors/kafka_sensor.py +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -12,9 +12,11 @@ import logging -from src.kafka_hook import KafkaConsumerHook -from airflow.operators.sensors import BaseSensorOperator -from airflow.utils import apply_defaults +from werkzeug.utils import cached_property + +from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class KafkaSensor(BaseSensorOperator): @@ -22,6 +24,8 @@ class KafkaSensor(BaseSensorOperator): Consumes the Kafka message with the specific topic """ + templated_fields = ('topic',) + @apply_defaults def __init__(self, conn_id, topic, *args, **kwargs): """ @@ -34,9 +38,13 @@ def __init__(self, conn_id, topic, *args, **kwargs): the subscribed topic """ self.topic = topic - self.hook = KafkaConsumerHook(conn_id, topic) + self.conn_id = conn_id super(KafkaSensor, self).__init__(*args, **kwargs) + @cached_property + def hook(self): + return KafkaConsumerHook(self.conn_id, self.topic) + def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', From 58b723985384e639b2160e1089ebb625c6505f8e Mon Sep 17 00:00:00 2001 From: dan Date: Mon, 2 Mar 2020 15:47:10 -0500 Subject: [PATCH 091/162] Moved files to providers package. Externalized connection logic from init function --- airflow/contrib/hooks/kafka_consumer_hook.py | 71 ------------------- airflow/contrib/hooks/kafka_producer_hook.py | 58 --------------- airflow/contrib/sensors/kafka_sensor.py | 58 --------------- airflow/providers/apache/kafka/__init__.py | 14 ++++ .../apache/kafka/example_dags/__init__.py | 14 ++++ .../kafka/example_dags/example_kafka_dag.py | 1 + .../providers/apache/kafka/hooks/__init__.py | 14 ++++ .../apache/kafka/hooks/kafka_consumer_hook.py | 47 ++++-------- .../apache/kafka/hooks/kafka_producer_hook.py | 65 +++++------------ .../apache/kafka/sensors/__init__.py | 14 ++++ .../apache/kafka/sensors/kafka_sensor.py | 57 ++++++--------- 11 files changed, 113 insertions(+), 300 deletions(-) delete mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py delete mode 100644 airflow/contrib/hooks/kafka_producer_hook.py delete mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py deleted file mode 100644 index 1143e2ccfb6a7..0000000000000 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ /dev/null @@ -1,71 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaConsumer - - -class KafkaConsumerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - # Disable auto commit as the hook will commit right - # after polling. - conf['enable_auto_commit'] = False - - self.server = f"""{host}:{port}""" - self.consumer = KafkaConsumer( - self.topic, - bootstrap_servers=self.server, **conf) - - return self.consumer - - def get_messages(self, timeout_ms=50): - """ - Get all the messages haven't been consumed, it doesn't - block by default, then commit the offset. - :return: - A list of messages - """ - consumer = self.get_conn() - try: - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms) - - consumer.commit() - finally: - consumer.close() - return messages - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.consumer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py deleted file mode 100644 index 9c1ebb158afe5..0000000000000 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaProducer - - -class KafkaProducerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaProducerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - conf['enable_auto_commit'] = False - self.server = f"""{host}:{port}""" - self.producer = KafkaProducer( - bootstrap_servers=self.server, **conf) - - return self.producer - - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - producer = self.get_conn() - try: - future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, - timestamp_ms=timestamp_ms) - finally: - producer.close() - return future_record_metadata - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.produer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py deleted file mode 100644 index e51328cdd63a9..0000000000000 --- a/airflow/contrib/sensors/kafka_sensor.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import logging - -from werkzeug.utils import cached_property - -from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook -from airflow.sensors.base_sensor_operator import BaseSensorOperator -from airflow.utils.decorators import apply_defaults - - -class KafkaSensor(BaseSensorOperator): - """ - Consumes the Kafka message with the specific topic - """ - - templated_fields = ('topic',) - - @apply_defaults - def __init__(self, conn_id, topic, *args, **kwargs): - """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - :param conn_id: - the kafka broker connection whom this sensor - subscripts against. - :param topic: - the subscribed topic - """ - self.topic = topic - self.conn_id = conn_id - super(KafkaSensor, self).__init__(*args, **kwargs) - - @cached_property - def hook(self): - return KafkaConsumerHook(self.conn_id, self.topic) - - def poke(self, context): - logging.info( - 'Poking topic: %s, using hook: %s', - self.topic, self.hook) - - messages = self.hook.get_messages() - - logging.info( - 'Got messages during poking: %s', messages) - - return messages or False diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py index 217e5db960782..e136811d94d06 100644 --- a/airflow/providers/apache/kafka/__init__.py +++ b/airflow/providers/apache/kafka/__init__.py @@ -1,3 +1,4 @@ +<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -15,3 +16,16 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +======= +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +>>>>>>> Moved files to providers package. Externalized connection logic from init function diff --git a/airflow/providers/apache/kafka/example_dags/__init__.py b/airflow/providers/apache/kafka/example_dags/__init__.py index 217e5db960782..e136811d94d06 100644 --- a/airflow/providers/apache/kafka/example_dags/__init__.py +++ b/airflow/providers/apache/kafka/example_dags/__init__.py @@ -1,3 +1,4 @@ +<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -15,3 +16,16 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +======= +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +>>>>>>> Moved files to providers package. Externalized connection logic from init function diff --git a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py index eb89df6b7cca8..0d718ba2c80cf 100644 --- a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py +++ b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py @@ -22,6 +22,7 @@ from airflow.utils.timezone import datetime from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor + DAG_ID = "example_kafka_dag" dag_start_date = datetime(2015, 6, 1, hour=20, tzinfo=None) default_args = { diff --git a/airflow/providers/apache/kafka/hooks/__init__.py b/airflow/providers/apache/kafka/hooks/__init__.py index 217e5db960782..e136811d94d06 100644 --- a/airflow/providers/apache/kafka/hooks/__init__.py +++ b/airflow/providers/apache/kafka/hooks/__init__.py @@ -1,3 +1,4 @@ +<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -15,3 +16,16 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +======= +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +>>>>>>> Moved files to providers package. Externalized connection logic from init function diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 6b58bd3091d1a..f647487b3bb05 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -1,30 +1,22 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at # -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 # -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from airflow.hooks import base_hook as BaseHook from kafka import KafkaConsumer from airflow.hooks.base_hook import BaseHook class KafkaConsumerHook(BaseHook): - """ - KafkaConsumerHook Class. - """ + DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 @@ -39,13 +31,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self) -> KafkaConsumer: - """ - A Kafka Consumer object. - - :return: - A Kafka Consumer object. - """ + def get_conn(self): if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -60,12 +46,10 @@ def get_conn(self) -> KafkaConsumer: ) return self.consumer - def get_messages(self, timeout_ms=5000) -> dict: + def get_messages(self, timeout_ms=5000): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. - - :param timeout_ms: :return: A list of messages """ @@ -79,10 +63,7 @@ def get_messages(self, timeout_ms=5000) -> dict: def __repr__(self): """ - A pretty version of the connection string. - - :return: - A pretty version of the connection string. + Pretty the hook with the connection info """ connected = self.consumer is not None return ' KafkaProducer: - """ - Returns a Kafka Producer - - :return: - A Kafka Producer object. - """ + def get_conn(self): if not self._conn: - _conn = self.get_connection(self.conn_id) - service_options = _conn.extra_dejson - host = _conn.host or self.DEFAULT_HOST - port = _conn.port or self.DEFAULT_PORT + conn = self.get_connection(self.conn_id) + service_options = conn.extra_dejson + host = conn.host or self.DEFAULT_HOST + port = conn.port or self.DEFAULT_PORT self.server = f"""{host}:{port}""" self.consumer = KafkaProducer( @@ -58,16 +44,6 @@ def get_conn(self) -> KafkaProducer: return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - """ - Sends a message on the specified topic and partition. Keyed messages will be sent in order. - - :param topic: - :param value: - :param key: - :param partition: - :param timestamp_ms: - :return: - """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, @@ -78,12 +54,9 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - A pretty version of the connection string. - - :return: - A pretty version of the connection string. + Pretty the hook with the connection info """ - connected = self.producer is not None + connected = self.produer is not None return '' % \ (connected, self.server, self.topic) diff --git a/airflow/providers/apache/kafka/sensors/__init__.py b/airflow/providers/apache/kafka/sensors/__init__.py index 217e5db960782..e136811d94d06 100644 --- a/airflow/providers/apache/kafka/sensors/__init__.py +++ b/airflow/providers/apache/kafka/sensors/__init__.py @@ -1,3 +1,4 @@ +<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -15,3 +16,16 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +======= +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +>>>>>>> Moved files to providers package. Externalized connection logic from init function diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 481c69a321557..f7043a0294706 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -1,20 +1,16 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at # -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 # -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import logging from cached_property import cached_property from airflow.utils.decorators import apply_defaults @@ -36,14 +32,13 @@ class KafkaSensor(BaseSensorOperator): @apply_defaults def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs): """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. :param topic: - :param host: - :param port: - :param args: - :param kwargs: + the subscribed topic """ self.topic = topic self.host = host @@ -52,24 +47,18 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): - """ - Returns a Kafka Consumer Hook - """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): - """ - Checks to see if messages exist on this topic/partition. - - :param context: - :return: - """ - self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) messages = self.hook.get_messages() - if messages: - self.log.info('Got messages during poking: %s', str(messages)) + if messages is not {}: + logging.info( + 'Got messages during poking: %s', messages) return messages else: return False From fb4e65406c919b5eac066e4e3e2280405be8a594 Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 11 Mar 2020 09:06:55 -0400 Subject: [PATCH 092/162] Fixing Travis CI Linting errors --- airflow/providers/apache/kafka/__init__.py | 22 ++++++++++++++++- .../providers/apache/kafka/hooks/__init__.py | 22 ++++++++++++++++- .../apache/kafka/hooks/kafka_consumer_hook.py | 1 - .../apache/kafka/hooks/kafka_producer_hook.py | 1 - .../apache/kafka/sensors/kafka_sensor.py | 24 ++++++++++++------- 5 files changed, 57 insertions(+), 13 deletions(-) diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py index e136811d94d06..15388643f828a 100644 --- a/airflow/providers/apache/kafka/__init__.py +++ b/airflow/providers/apache/kafka/__init__.py @@ -1,4 +1,5 @@ <<<<<<< HEAD +<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -20,12 +21,31 @@ # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. # You may obtain a copy of the License at +======= +>>>>>>> Fixing Travis CI Linting errors # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # +<<<<<<< HEAD # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. >>>>>>> Moved files to providers package. Externalized connection logic from init function +======= +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +>>>>>>> Fixing Travis CI Linting errors diff --git a/airflow/providers/apache/kafka/hooks/__init__.py b/airflow/providers/apache/kafka/hooks/__init__.py index e136811d94d06..15388643f828a 100644 --- a/airflow/providers/apache/kafka/hooks/__init__.py +++ b/airflow/providers/apache/kafka/hooks/__init__.py @@ -1,4 +1,5 @@ <<<<<<< HEAD +<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -20,12 +21,31 @@ # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. # You may obtain a copy of the License at +======= +>>>>>>> Fixing Travis CI Linting errors # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # +<<<<<<< HEAD # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. >>>>>>> Moved files to providers package. Externalized connection logic from init function +======= +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +>>>>>>> Fixing Travis CI Linting errors diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index f647487b3bb05..968514398a20b 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -10,7 +10,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -from airflow.hooks import base_hook as BaseHook from kafka import KafkaConsumer from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 0ef22b09b35b9..22c069e17b080 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -10,7 +10,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -from airflow.hooks import base_hook as BaseHook from kafka import KafkaProducer from kafka.producer.future import FutureRecordMetadata from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index f7043a0294706..c5096d8734fcd 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -1,14 +1,20 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. import logging From 677e23fae97f320513f2b04f4f808b934aabf220 Mon Sep 17 00:00:00 2001 From: dan Date: Thu, 30 Apr 2020 09:57:14 -0400 Subject: [PATCH 093/162] Fixed isort issue --- airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py | 1 + airflow/providers/apache/kafka/sensors/kafka_sensor.py | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 968514398a20b..fcdeb45664abe 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -11,6 +11,7 @@ # limitations under the License. from kafka import KafkaConsumer + from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index c5096d8734fcd..8649f876407ec 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -19,9 +19,10 @@ import logging from cached_property import cached_property -from airflow.utils.decorators import apply_defaults -from airflow.sensors.base_sensor_operator import BaseSensorOperator + from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class KafkaSensor(BaseSensorOperator): From 4972f304d2a12d2246ec50e18e84ec6b7f71aedb Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:16:08 -0400 Subject: [PATCH 094/162] produer --> producer --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 22c069e17b080..66fe3a057a730 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -55,7 +55,7 @@ def __repr__(self): """ Pretty the hook with the connection info """ - connected = self.produer is not None + connected = self.producer is not None return '' % \ (connected, self.server, self.topic) From 06cae89144b9ca9f7dd350d9489ec08f2f59b46a Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:36:16 -0400 Subject: [PATCH 095/162] Added pylint mandated doc strings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 2 ++ .../apache/kafka/hooks/kafka_producer_hook.py | 11 +++++++++++ .../providers/apache/kafka/sensors/kafka_sensor.py | 7 ++++++- 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index fcdeb45664abe..ed06d2965e6d1 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -16,7 +16,9 @@ class KafkaConsumerHook(BaseHook): + """ + """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 66fe3a057a730..910eb9d0d09ea 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -16,7 +16,9 @@ class KafkaProducerHook(BaseHook): + """ + """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 @@ -43,6 +45,15 @@ def get_conn(self): return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + """ + + :param topic: + :param value: + :param key: + :param partition: + :param timestamp_ms: + :return: + """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 8649f876407ec..e4c33f5fa16f8 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -54,12 +54,17 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): + """ + + :return: + KafkaConsumerHook + """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', - self.topic, self.hook) + str(self.topic), str(self.hook)) messages = self.hook.get_messages() From 6c5a96e8ced3357d8cb59f0ed230a0f3dc73a1e6 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 10:06:05 -0400 Subject: [PATCH 096/162] Fixing more pylint issues --- airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py | 2 +- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index ed06d2965e6d1..b1e77af540415 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -17,7 +17,7 @@ class KafkaConsumerHook(BaseHook): """ - + KafkaConsumerHook Class. """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 910eb9d0d09ea..5339c67a5f92c 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -17,7 +17,7 @@ class KafkaProducerHook(BaseHook): """ - + KafkaProducerHook Class. """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index e4c33f5fa16f8..250656398ac4e 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -68,9 +68,9 @@ def poke(self, context): messages = self.hook.get_messages() - if messages is not {}: + if not messages: logging.info( - 'Got messages during poking: %s', messages) + 'Got messages during poking: %s', str(messages)) return messages else: return False From d8f1106743a2442b2966f6de1ce24cd5dff12082 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 097/162] Tried to fix docstrings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 17 +++++++++++--- .../apache/kafka/hooks/kafka_producer_hook.py | 16 +++++++++++--- .../apache/kafka/sensors/kafka_sensor.py | 22 +++++++++++++------ 3 files changed, 42 insertions(+), 13 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index b1e77af540415..f178fc123ae52 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -33,7 +33,13 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self): + def get_conn(self) -> KafkaConsumer: + """ + A Kafka Consumer object. + + :return: + A Kafka Consumer object. + """ if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -48,10 +54,12 @@ def get_conn(self): ) return self.consumer - def get_messages(self, timeout_ms=5000): + def get_messages(self, timeout_ms=5000) -> dict: """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. + + :param timeout_ms: :return: A list of messages """ @@ -65,7 +73,10 @@ def get_messages(self, timeout_ms=5000): def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.consumer is not None return ' KafkaProducer: + """ + Returns a Kafka Producer + + :return: + A Kafka Producer object. + """ if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -44,8 +50,9 @@ def get_conn(self): ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: """ + Sends a message on the specified topic and partition. Keyed messages will be sent in order. :param topic: :param value: @@ -64,7 +71,10 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.producer is not None return ' Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 098/162] Tried to fix docstrings --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 48598216060a3..65102ab73ac9f 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -50,7 +50,7 @@ def get_conn(self) -> KafkaProducer: ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): """ Sends a message on the specified topic and partition. Keyed messages will be sent in order. From 9de53ae671cf2ad80c65ad02742f05f7a4140691 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 9 May 2020 23:41:58 +0100 Subject: [PATCH 099/162] Apply suggestions from code review --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 50ca55079d9ac..60b95989b64cb 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -56,10 +56,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): """ - Returns a Kafka Consumer Hook - - :return: - KafkaConsumerHook + Returns a Kafka Consumer Hook """ return KafkaConsumerHook(self.topic, self.host, self.port) From 080c3b796ae7ae97a75fbc4371e538b4e5988415 Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:28 -0400 Subject: [PATCH 100/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 60b95989b64cb..7108f84207bcd 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -74,8 +74,7 @@ def poke(self, context): messages = self.hook.get_messages() if messages: - logging.info( - 'Got messages during poking: %s', str(messages)) + self.log.info('Got messages during poking: %s', str(messages)) return messages else: return False From 34482236838bcc4dad12bcd8699baaa1aa7d1f8a Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:46 -0400 Subject: [PATCH 101/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 7108f84207bcd..bb27e2f5adc73 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -67,9 +67,7 @@ def poke(self, context): :param context: :return: """ - logging.info( - 'Poking topic: %s, using hook: %s', - str(self.topic), str(self.hook)) + self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) messages = self.hook.get_messages() From 40bfe9ed704f362bbfe4a3f5c6825500d5ac2681 Mon Sep 17 00:00:00 2001 From: dan Date: Sat, 23 May 2020 14:46:59 -0400 Subject: [PATCH 102/162] Removed unused imports --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index bb27e2f5adc73..842643c60ba40 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -16,8 +16,6 @@ # specific language governing permissions and limitations # under the License. -import logging - from cached_property import cached_property from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook From c4c6e1447f70208d202bb8baeafec6eceb51fffd Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 14:21:35 -0400 Subject: [PATCH 103/162] Modifying readme to trigger builds --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 647b9f9ac4599..c2869acb4661f 100644 --- a/README.md +++ b/README.md @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. --> - # Apache Airflow [![PyPI version](https://badge.fury.io/py/apache-airflow.svg)](https://badge.fury.io/py/apache-airflow) From 6d018f4e367c8a7d38d4e1b7b2ff4a2f3187411b Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 14:30:56 -0400 Subject: [PATCH 104/162] Revert "Resolved conflicts in requirements files" This reverts commit 9b213d23ec86d4875dddc9b2dbc7c5a0c5f1f93b. --- requirements/requirements-python3.6.txt | 7 ++++--- requirements/requirements-python3.7.txt | 4 ++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index fed4f599bec4b..321465eae5ac4 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -56,7 +56,7 @@ azure-batch==9.0.0 azure-common==1.1.25 azure-cosmos==3.2.0 azure-datalake-store==0.0.48 -azure-kusto-data==0.1.0 +azure-kusto-data==0.0.45 azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 @@ -138,6 +138,7 @@ gcsfs==0.6.2 gevent==20.6.2 gitdb==4.0.5 google-ads==4.0.0 +<<<<<<< HEAD google-api-core==1.21.0 google-api-python-client==1.9.3 google-auth-httplib2==0.0.4 @@ -163,7 +164,7 @@ google-cloud-spanner==1.17.1 google-cloud-speech==1.3.2 google-cloud-storage==1.29.0 google-cloud-tasks==1.5.0 -google-cloud-texttospeech==2.0.0 +google-cloud-texttospeech==1.0.1 google-cloud-translate==2.0.1 google-cloud-videointelligence==1.15.0 google-cloud-vision==1.0.0 @@ -392,7 +393,7 @@ virtualenv==20.0.26 watchtower==0.7.3 wcwidth==0.2.5 websocket-client==0.57.0 -wrapt==1.12.1 +wrapt==1.11.2 xmltodict==0.12.0 yamllint==1.23.0 yandexcloud==0.43.0 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index 6298069a2faf2..ced25f19a306a 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -56,7 +56,7 @@ azure-batch==9.0.0 azure-common==1.1.25 azure-cosmos==3.2.0 azure-datalake-store==0.0.48 -azure-kusto-data==0.1.0 +azure-kusto-data==0.0.45 azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 @@ -162,7 +162,7 @@ google-cloud-spanner==1.17.1 google-cloud-speech==1.3.2 google-cloud-storage==1.29.0 google-cloud-tasks==1.5.0 -google-cloud-texttospeech==2.0.0 +google-cloud-texttospeech==1.0.1 google-cloud-translate==2.0.1 google-cloud-videointelligence==1.15.0 google-cloud-vision==1.0.0 From d63294923fc7a494efa3277f7b54ade61512adf9 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 14:52:39 -0400 Subject: [PATCH 105/162] Updated readme to trigger build --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index c2869acb4661f..647b9f9ac4599 100644 --- a/README.md +++ b/README.md @@ -16,6 +16,7 @@ specific language governing permissions and limitations under the License. --> + # Apache Airflow [![PyPI version](https://badge.fury.io/py/apache-airflow.svg)](https://badge.fury.io/py/apache-airflow) From 7a40db4ce50ec00ee81bfc2d4f0ad332b7d94d77 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 15:20:54 -0400 Subject: [PATCH 106/162] updating requirements files --- requirements/requirements-python3.6.txt | 5 +---- requirements/requirements-python3.7.txt | 3 --- 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index 321465eae5ac4..3c7ee9b2b3896 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -138,7 +138,6 @@ gcsfs==0.6.2 gevent==20.6.2 gitdb==4.0.5 google-ads==4.0.0 -<<<<<<< HEAD google-api-core==1.21.0 google-api-python-client==1.9.3 google-auth-httplib2==0.0.4 @@ -214,7 +213,6 @@ kubernetes==11.0.0 kylinpy==2.8.1 lazy-object-proxy==1.5.0 ldap3==2.7 -libcst==0.3.6 lockfile==0.12.2 marshmallow-enum==1.5.1 marshmallow-oneofschema==2.0.1 @@ -268,7 +266,6 @@ presto-python-client==0.7.0 prison==0.1.3 prometheus-client==0.8.0 prompt-toolkit==3.0.5 -proto-plus==0.4.0 protobuf==3.12.2 psutil==5.7.0 psycopg2-binary==2.8.5 @@ -393,7 +390,7 @@ virtualenv==20.0.26 watchtower==0.7.3 wcwidth==0.2.5 websocket-client==0.57.0 -wrapt==1.11.2 +wrapt==1.12.1 xmltodict==0.12.0 yamllint==1.23.0 yandexcloud==0.43.0 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index ced25f19a306a..28763d28cd629 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -210,7 +210,6 @@ kubernetes==11.0.0 kylinpy==2.8.1 lazy-object-proxy==1.5.0 ldap3==2.7 -libcst==0.3.6 lockfile==0.12.2 marshmallow-enum==1.5.1 marshmallow-oneofschema==2.0.1 @@ -263,7 +262,6 @@ presto-python-client==0.7.0 prison==0.1.3 prometheus-client==0.8.0 prompt-toolkit==3.0.5 -proto-plus==0.4.0 protobuf==3.12.2 psutil==5.7.0 psycopg2-binary==2.8.5 @@ -375,7 +373,6 @@ tqdm==4.47.0 traitlets==4.3.3 typed-ast==1.4.1 typing-extensions==3.7.4.2 -typing-inspect==0.6.0 tzlocal==1.5.1 unicodecsv==0.14.1 uritemplate==3.0.1 From e933030315a69fb797a442045d0d5f6cd7e860b5 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 15:52:00 -0400 Subject: [PATCH 107/162] updated requirements --- requirements/requirements-python3.7.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index 28763d28cd629..ae515b109b793 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -383,7 +383,7 @@ virtualenv==20.0.26 watchtower==0.7.3 wcwidth==0.2.5 websocket-client==0.57.0 -wrapt==1.11.2 +wrapt==1.12.1 xmltodict==0.12.0 yamllint==1.23.0 yandexcloud==0.43.0 From 0623f7ace16a8c006dfa06625dff896efc0a98f8 Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 12 Feb 2020 13:51:03 -0500 Subject: [PATCH 108/162] Add KafkaConsumerHook, KafkaProduerHook and KafkaSensor MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Dan Ferguson dferguson992@gmail.com Co-authored-by: Hanan Shteingart Co-authored-by: YuanfΞi Zhu --- airflow/contrib/hooks/kafka_consumer_hook.py | 69 ++++++++++++++++++++ airflow/contrib/hooks/kafka_producer_hook.py | 56 ++++++++++++++++ airflow/contrib/sensors/kafka_sensor.py | 50 ++++++++++++++ 3 files changed, 175 insertions(+) create mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py create mode 100644 airflow/contrib/hooks/kafka_producer_hook.py create mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py new file mode 100644 index 0000000000000..335c267b7837f --- /dev/null +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -0,0 +1,69 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaConsumer + + +class KafkaConsumerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + # Disable auto commit as the hook will commit right + # after polling. + conf['enable_auto_commit'] = False + + self.server = '{host}:{port}'.format(**locals()) + self.consumer = KafkaConsumer( + self.topic, + bootstrap_servers=self.server, **conf) + + return self.consumer + + def get_messages(self): + """ + Get all the messages haven't been consumed, it doesn't + block by default, then commit the offset. + :return: + A list of messages + """ + consumer = self.get_conn() + + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms=50) + + consumer.commit() + return messages + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.consumer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py new file mode 100644 index 0000000000000..ffa5d4693e9c2 --- /dev/null +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -0,0 +1,56 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaProducer + + +class KafkaProducerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + conf['enable_auto_commit'] = False + self.server = '{host}:{port}'.format(**locals()) + self.producer = KafkaProducer( + bootstrap_servers=self.server, **conf) + + return self.producer + + def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): + producer = self.get_pub() + future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, + timestamp_ms=timestamp_ms) + producer.close() + return future_record_metadata + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.produer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py new file mode 100644 index 0000000000000..738e63c76c730 --- /dev/null +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -0,0 +1,50 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import logging + +from src.kafka_hook import KafkaConsumerHook +from airflow.operators.sensors import BaseSensorOperator +from airflow.utils import apply_defaults + + +class KafkaSensor(BaseSensorOperator): + """ + Consumes the Kafka message with the specific topic + """ + + @apply_defaults + def __init__(self, conn_id, topic, *args, **kwargs): + """ + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. + :param topic: + the subscribed topic + """ + self.topic = topic + self.hook = KafkaConsumerHook(conn_id, topic) + super(KafkaSensor, self).__init__(*args, **kwargs) + + def poke(self, context): + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) + + messages = self.hook.get_messages() + + logging.info( + 'Got messages during poking: %s', messages) + + return messages or False From 707b2572780ceefcf89738fcd91ff60cb505ceee Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 18 Feb 2020 09:05:42 -0500 Subject: [PATCH 109/162] Modifying files in accordance with peer review --- airflow/contrib/hooks/kafka_consumer_hook.py | 24 +++++++++++--------- airflow/contrib/hooks/kafka_producer_hook.py | 24 +++++++++++--------- airflow/contrib/sensors/kafka_sensor.py | 16 +++++++++---- 3 files changed, 38 insertions(+), 26 deletions(-) diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py index 335c267b7837f..1143e2ccfb6a7 100644 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -16,8 +16,8 @@ class KafkaConsumerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): super(KafkaConsumerHook, self).__init__(None) @@ -29,21 +29,21 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT # Disable auto commit as the hook will commit right # after polling. conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.consumer = KafkaConsumer( self.topic, bootstrap_servers=self.server, **conf) return self.consumer - def get_messages(self): + def get_messages(self, timeout_ms=50): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. @@ -51,12 +51,14 @@ def get_messages(self): A list of messages """ consumer = self.get_conn() + try: + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms) - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms=50) - - consumer.commit() + consumer.commit() + finally: + consumer.close() return messages def __repr__(self): diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py index ffa5d4693e9c2..9c1ebb158afe5 100644 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -16,11 +16,11 @@ class KafkaProducerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) + super(KafkaProducerHook, self).__init__(None) self.conn = self.get_connection(conn_id) self.server = None self.consumer = None @@ -29,21 +29,23 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.producer = KafkaProducer( bootstrap_servers=self.server, **conf) return self.producer - def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): - producer = self.get_pub() - future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, - timestamp_ms=timestamp_ms) - producer.close() + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + producer = self.get_conn() + try: + future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, + timestamp_ms=timestamp_ms) + finally: + producer.close() return future_record_metadata def __repr__(self): diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py index 738e63c76c730..e51328cdd63a9 100644 --- a/airflow/contrib/sensors/kafka_sensor.py +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -12,9 +12,11 @@ import logging -from src.kafka_hook import KafkaConsumerHook -from airflow.operators.sensors import BaseSensorOperator -from airflow.utils import apply_defaults +from werkzeug.utils import cached_property + +from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class KafkaSensor(BaseSensorOperator): @@ -22,6 +24,8 @@ class KafkaSensor(BaseSensorOperator): Consumes the Kafka message with the specific topic """ + templated_fields = ('topic',) + @apply_defaults def __init__(self, conn_id, topic, *args, **kwargs): """ @@ -34,9 +38,13 @@ def __init__(self, conn_id, topic, *args, **kwargs): the subscribed topic """ self.topic = topic - self.hook = KafkaConsumerHook(conn_id, topic) + self.conn_id = conn_id super(KafkaSensor, self).__init__(*args, **kwargs) + @cached_property + def hook(self): + return KafkaConsumerHook(self.conn_id, self.topic) + def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', From 45229af5648a2c50493de5b413ee86ec5062ffaf Mon Sep 17 00:00:00 2001 From: dan Date: Mon, 2 Mar 2020 15:47:10 -0500 Subject: [PATCH 110/162] Moved files to providers package. Externalized connection logic from init function --- airflow/contrib/hooks/kafka_consumer_hook.py | 71 ------------------- airflow/contrib/hooks/kafka_producer_hook.py | 58 --------------- airflow/contrib/sensors/kafka_sensor.py | 58 --------------- airflow/providers/apache/kafka/__init__.py | 34 --------- .../apache/kafka/example_dags/__init__.py | 14 ---- .../providers/apache/kafka/hooks/__init__.py | 34 --------- .../apache/kafka/hooks/kafka_consumer_hook.py | 24 ++----- .../apache/kafka/hooks/kafka_producer_hook.py | 30 ++------ .../apache/kafka/sensors/__init__.py | 14 ---- .../apache/kafka/sensors/kafka_sensor.py | 62 +++++++--------- 10 files changed, 35 insertions(+), 364 deletions(-) delete mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py delete mode 100644 airflow/contrib/hooks/kafka_producer_hook.py delete mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py deleted file mode 100644 index 1143e2ccfb6a7..0000000000000 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ /dev/null @@ -1,71 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaConsumer - - -class KafkaConsumerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - # Disable auto commit as the hook will commit right - # after polling. - conf['enable_auto_commit'] = False - - self.server = f"""{host}:{port}""" - self.consumer = KafkaConsumer( - self.topic, - bootstrap_servers=self.server, **conf) - - return self.consumer - - def get_messages(self, timeout_ms=50): - """ - Get all the messages haven't been consumed, it doesn't - block by default, then commit the offset. - :return: - A list of messages - """ - consumer = self.get_conn() - try: - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms) - - consumer.commit() - finally: - consumer.close() - return messages - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.consumer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py deleted file mode 100644 index 9c1ebb158afe5..0000000000000 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaProducer - - -class KafkaProducerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaProducerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - conf['enable_auto_commit'] = False - self.server = f"""{host}:{port}""" - self.producer = KafkaProducer( - bootstrap_servers=self.server, **conf) - - return self.producer - - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - producer = self.get_conn() - try: - future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, - timestamp_ms=timestamp_ms) - finally: - producer.close() - return future_record_metadata - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.produer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py deleted file mode 100644 index e51328cdd63a9..0000000000000 --- a/airflow/contrib/sensors/kafka_sensor.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import logging - -from werkzeug.utils import cached_property - -from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook -from airflow.sensors.base_sensor_operator import BaseSensorOperator -from airflow.utils.decorators import apply_defaults - - -class KafkaSensor(BaseSensorOperator): - """ - Consumes the Kafka message with the specific topic - """ - - templated_fields = ('topic',) - - @apply_defaults - def __init__(self, conn_id, topic, *args, **kwargs): - """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - :param conn_id: - the kafka broker connection whom this sensor - subscripts against. - :param topic: - the subscribed topic - """ - self.topic = topic - self.conn_id = conn_id - super(KafkaSensor, self).__init__(*args, **kwargs) - - @cached_property - def hook(self): - return KafkaConsumerHook(self.conn_id, self.topic) - - def poke(self, context): - logging.info( - 'Poking topic: %s, using hook: %s', - self.topic, self.hook) - - messages = self.hook.get_messages() - - logging.info( - 'Got messages during poking: %s', messages) - - return messages or False diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py index 15388643f828a..217e5db960782 100644 --- a/airflow/providers/apache/kafka/__init__.py +++ b/airflow/providers/apache/kafka/__init__.py @@ -1,5 +1,3 @@ -<<<<<<< HEAD -<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -17,35 +15,3 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -======= -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -======= ->>>>>>> Fixing Travis CI Linting errors -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -<<<<<<< HEAD -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. ->>>>>>> Moved files to providers package. Externalized connection logic from init function -======= -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. ->>>>>>> Fixing Travis CI Linting errors diff --git a/airflow/providers/apache/kafka/example_dags/__init__.py b/airflow/providers/apache/kafka/example_dags/__init__.py index e136811d94d06..217e5db960782 100644 --- a/airflow/providers/apache/kafka/example_dags/__init__.py +++ b/airflow/providers/apache/kafka/example_dags/__init__.py @@ -1,4 +1,3 @@ -<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -16,16 +15,3 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -======= -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. ->>>>>>> Moved files to providers package. Externalized connection logic from init function diff --git a/airflow/providers/apache/kafka/hooks/__init__.py b/airflow/providers/apache/kafka/hooks/__init__.py index 15388643f828a..217e5db960782 100644 --- a/airflow/providers/apache/kafka/hooks/__init__.py +++ b/airflow/providers/apache/kafka/hooks/__init__.py @@ -1,5 +1,3 @@ -<<<<<<< HEAD -<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -17,35 +15,3 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -======= -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -======= ->>>>>>> Fixing Travis CI Linting errors -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -<<<<<<< HEAD -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. ->>>>>>> Moved files to providers package. Externalized connection logic from init function -======= -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. ->>>>>>> Fixing Travis CI Linting errors diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index f178fc123ae52..cbb2a4c963839 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -10,15 +10,12 @@ # See the License for the specific language governing permissions and # limitations under the License. +from airflow.hooks import base_hook as BaseHook from kafka import KafkaConsumer -from airflow.hooks.base_hook import BaseHook - class KafkaConsumerHook(BaseHook): - """ - KafkaConsumerHook Class. - """ + DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 @@ -33,13 +30,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self) -> KafkaConsumer: - """ - A Kafka Consumer object. - - :return: - A Kafka Consumer object. - """ + def get_conn(self): if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -54,12 +45,10 @@ def get_conn(self) -> KafkaConsumer: ) return self.consumer - def get_messages(self, timeout_ms=5000) -> dict: + def get_messages(self, timeout_ms=5000): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. - - :param timeout_ms: :return: A list of messages """ @@ -73,10 +62,7 @@ def get_messages(self, timeout_ms=5000) -> dict: def __repr__(self): """ - A pretty version of the connection string. - - :return: - A pretty version of the connection string. + Pretty the hook with the connection info """ connected = self.consumer is not None return ' KafkaProducer: - """ - Returns a Kafka Producer - - :return: - A Kafka Producer object. - """ + def get_conn(self): if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -51,16 +44,6 @@ def get_conn(self) -> KafkaProducer: return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - """ - Sends a message on the specified topic and partition. Keyed messages will be sent in order. - - :param topic: - :param value: - :param key: - :param partition: - :param timestamp_ms: - :return: - """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, @@ -71,12 +54,9 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - A pretty version of the connection string. - - :return: - A pretty version of the connection string. + Pretty the hook with the connection info """ - connected = self.producer is not None + connected = self.produer is not None return '' % \ (connected, self.server, self.topic) diff --git a/airflow/providers/apache/kafka/sensors/__init__.py b/airflow/providers/apache/kafka/sensors/__init__.py index e136811d94d06..217e5db960782 100644 --- a/airflow/providers/apache/kafka/sensors/__init__.py +++ b/airflow/providers/apache/kafka/sensors/__init__.py @@ -1,4 +1,3 @@ -<<<<<<< HEAD # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -16,16 +15,3 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -======= -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. ->>>>>>> Moved files to providers package. Externalized connection logic from init function diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 842643c60ba40..f7043a0294706 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -1,26 +1,21 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at # -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 # -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. -from cached_property import cached_property +import logging -from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook -from airflow.sensors.base_sensor_operator import BaseSensorOperator +from cached_property import cached_property from airflow.utils.decorators import apply_defaults +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook class KafkaSensor(BaseSensorOperator): @@ -37,14 +32,13 @@ class KafkaSensor(BaseSensorOperator): @apply_defaults def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs): """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. :param topic: - :param host: - :param port: - :param args: - :param kwargs: + the subscribed topic """ self.topic = topic self.host = host @@ -53,24 +47,18 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): - """ - Returns a Kafka Consumer Hook - """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): - """ - Checks to see if messages exist on this topic/partition. - - :param context: - :return: - """ - self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) messages = self.hook.get_messages() - if messages: - self.log.info('Got messages during poking: %s', str(messages)) + if messages is not {}: + logging.info( + 'Got messages during poking: %s', messages) return messages else: return False From 7e543756bd82dd5c04e6f81fb9dc20c5ab5af8eb Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:16:08 -0400 Subject: [PATCH 111/162] produer --> producer --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 0ef22b09b35b9..83334444dafe3 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -56,7 +56,7 @@ def __repr__(self): """ Pretty the hook with the connection info """ - connected = self.produer is not None + connected = self.producer is not None return '' % \ (connected, self.server, self.topic) From a01f28c9ba5bf9748dfe8e0c805e87dc4a98b591 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:36:16 -0400 Subject: [PATCH 112/162] Added pylint mandated doc strings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 2 ++ .../apache/kafka/hooks/kafka_producer_hook.py | 11 +++++++++++ .../providers/apache/kafka/sensors/kafka_sensor.py | 7 ++++++- 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index cbb2a4c963839..19e039995b810 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -15,7 +15,9 @@ class KafkaConsumerHook(BaseHook): + """ + """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 83334444dafe3..0a94ad0db28b6 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -17,7 +17,9 @@ class KafkaProducerHook(BaseHook): + """ + """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 @@ -44,6 +46,15 @@ def get_conn(self): return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + """ + + :param topic: + :param value: + :param key: + :param partition: + :param timestamp_ms: + :return: + """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index f7043a0294706..65cd64af71b92 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -47,12 +47,17 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): + """ + + :return: + KafkaConsumerHook + """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', - self.topic, self.hook) + str(self.topic), str(self.hook)) messages = self.hook.get_messages() From 4a7fbd2cd11f31cc518b9d9d43fd75d7509ba443 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 10:06:05 -0400 Subject: [PATCH 113/162] Fixing more pylint issues --- airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py | 2 +- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 19e039995b810..17ea7f55c79ec 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -16,7 +16,7 @@ class KafkaConsumerHook(BaseHook): """ - + KafkaConsumerHook Class. """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 0a94ad0db28b6..840f292adc68a 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -18,7 +18,7 @@ class KafkaProducerHook(BaseHook): """ - + KafkaProducerHook Class. """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 65cd64af71b92..1c7ef43137f88 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -61,9 +61,9 @@ def poke(self, context): messages = self.hook.get_messages() - if messages is not {}: + if not messages: logging.info( - 'Got messages during poking: %s', messages) + 'Got messages during poking: %s', str(messages)) return messages else: return False From 4eea9ad0f80a0ba677c0ee587211bca827d4ede6 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 114/162] Tried to fix docstrings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 17 +++++++++++--- .../apache/kafka/hooks/kafka_producer_hook.py | 16 +++++++++++--- .../apache/kafka/sensors/kafka_sensor.py | 22 +++++++++++++------ 3 files changed, 42 insertions(+), 13 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 17ea7f55c79ec..96472934ab63b 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -32,7 +32,13 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self): + def get_conn(self) -> KafkaConsumer: + """ + A Kafka Consumer object. + + :return: + A Kafka Consumer object. + """ if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -47,10 +53,12 @@ def get_conn(self): ) return self.consumer - def get_messages(self, timeout_ms=5000): + def get_messages(self, timeout_ms=5000) -> dict: """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. + + :param timeout_ms: :return: A list of messages """ @@ -64,7 +72,10 @@ def get_messages(self, timeout_ms=5000): def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.consumer is not None return ' KafkaProducer: + """ + Returns a Kafka Producer + + :return: + A Kafka Producer object. + """ if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -45,8 +51,9 @@ def get_conn(self): ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: """ + Sends a message on the specified topic and partition. Keyed messages will be sent in order. :param topic: :param value: @@ -65,7 +72,10 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.producer is not None return ' Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 115/162] Tried to fix docstrings --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index e146a438d70fc..8b58392ecf54a 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -51,7 +51,7 @@ def get_conn(self) -> KafkaProducer: ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): """ Sends a message on the specified topic and partition. Keyed messages will be sent in order. From 0676415c373f1868e162fd71242f09722ba388c7 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 9 May 2020 23:41:58 +0100 Subject: [PATCH 116/162] Apply suggestions from code review --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 1a82ac5618ead..3a8f2bde8d210 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -49,10 +49,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): """ - Returns a Kafka Consumer Hook - - :return: - KafkaConsumerHook + Returns a Kafka Consumer Hook """ return KafkaConsumerHook(self.topic, self.host, self.port) From 359666cc05b95937899ffc233a8fb8e9512d3af5 Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:28 -0400 Subject: [PATCH 117/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 3a8f2bde8d210..232d40eb19b7d 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -67,8 +67,7 @@ def poke(self, context): messages = self.hook.get_messages() if messages: - logging.info( - 'Got messages during poking: %s', str(messages)) + self.log.info('Got messages during poking: %s', str(messages)) return messages else: return False From 307dce5ef64707ad13afd43e377777679c27e243 Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:46 -0400 Subject: [PATCH 118/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 232d40eb19b7d..61f0c6ad784e1 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -60,9 +60,7 @@ def poke(self, context): :param context: :return: """ - logging.info( - 'Poking topic: %s, using hook: %s', - str(self.topic), str(self.hook)) + self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) messages = self.hook.get_messages() From f036ee6c626d11b5d972da7e70121776309d9337 Mon Sep 17 00:00:00 2001 From: dan Date: Sat, 23 May 2020 14:46:59 -0400 Subject: [PATCH 119/162] Removed unused imports --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 61f0c6ad784e1..46503c838d1b5 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -10,8 +10,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -import logging - from cached_property import cached_property from airflow.utils.decorators import apply_defaults from airflow.sensors.base_sensor_operator import BaseSensorOperator From b82cd1a38e75f5df6f88ffad93bb8e3ece3e5a6a Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 12 Feb 2020 13:51:03 -0500 Subject: [PATCH 120/162] Add KafkaConsumerHook, KafkaProduerHook and KafkaSensor MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Dan Ferguson dferguson992@gmail.com Co-authored-by: Hanan Shteingart Co-authored-by: YuanfΞi Zhu --- airflow/contrib/hooks/kafka_consumer_hook.py | 69 ++++++++++++++++++++ airflow/contrib/hooks/kafka_producer_hook.py | 56 ++++++++++++++++ airflow/contrib/sensors/kafka_sensor.py | 50 ++++++++++++++ 3 files changed, 175 insertions(+) create mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py create mode 100644 airflow/contrib/hooks/kafka_producer_hook.py create mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py new file mode 100644 index 0000000000000..335c267b7837f --- /dev/null +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -0,0 +1,69 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaConsumer + + +class KafkaConsumerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + # Disable auto commit as the hook will commit right + # after polling. + conf['enable_auto_commit'] = False + + self.server = '{host}:{port}'.format(**locals()) + self.consumer = KafkaConsumer( + self.topic, + bootstrap_servers=self.server, **conf) + + return self.consumer + + def get_messages(self): + """ + Get all the messages haven't been consumed, it doesn't + block by default, then commit the offset. + :return: + A list of messages + """ + consumer = self.get_conn() + + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms=50) + + consumer.commit() + return messages + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.consumer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py new file mode 100644 index 0000000000000..ffa5d4693e9c2 --- /dev/null +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -0,0 +1,56 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaProducer + + +class KafkaProducerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + conf['enable_auto_commit'] = False + self.server = '{host}:{port}'.format(**locals()) + self.producer = KafkaProducer( + bootstrap_servers=self.server, **conf) + + return self.producer + + def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): + producer = self.get_pub() + future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, + timestamp_ms=timestamp_ms) + producer.close() + return future_record_metadata + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.produer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py new file mode 100644 index 0000000000000..738e63c76c730 --- /dev/null +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -0,0 +1,50 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import logging + +from src.kafka_hook import KafkaConsumerHook +from airflow.operators.sensors import BaseSensorOperator +from airflow.utils import apply_defaults + + +class KafkaSensor(BaseSensorOperator): + """ + Consumes the Kafka message with the specific topic + """ + + @apply_defaults + def __init__(self, conn_id, topic, *args, **kwargs): + """ + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. + :param topic: + the subscribed topic + """ + self.topic = topic + self.hook = KafkaConsumerHook(conn_id, topic) + super(KafkaSensor, self).__init__(*args, **kwargs) + + def poke(self, context): + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) + + messages = self.hook.get_messages() + + logging.info( + 'Got messages during poking: %s', messages) + + return messages or False From 947a3c597b55a1cb686cfacd9b002b3d68720289 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 18 Feb 2020 09:05:42 -0500 Subject: [PATCH 121/162] Modifying files in accordance with peer review --- airflow/contrib/hooks/kafka_consumer_hook.py | 24 +++++++++++--------- airflow/contrib/hooks/kafka_producer_hook.py | 24 +++++++++++--------- airflow/contrib/sensors/kafka_sensor.py | 16 +++++++++---- 3 files changed, 38 insertions(+), 26 deletions(-) diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py index 335c267b7837f..1143e2ccfb6a7 100644 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -16,8 +16,8 @@ class KafkaConsumerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): super(KafkaConsumerHook, self).__init__(None) @@ -29,21 +29,21 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT # Disable auto commit as the hook will commit right # after polling. conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.consumer = KafkaConsumer( self.topic, bootstrap_servers=self.server, **conf) return self.consumer - def get_messages(self): + def get_messages(self, timeout_ms=50): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. @@ -51,12 +51,14 @@ def get_messages(self): A list of messages """ consumer = self.get_conn() + try: + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms) - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms=50) - - consumer.commit() + consumer.commit() + finally: + consumer.close() return messages def __repr__(self): diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py index ffa5d4693e9c2..9c1ebb158afe5 100644 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -16,11 +16,11 @@ class KafkaProducerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) + super(KafkaProducerHook, self).__init__(None) self.conn = self.get_connection(conn_id) self.server = None self.consumer = None @@ -29,21 +29,23 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.producer = KafkaProducer( bootstrap_servers=self.server, **conf) return self.producer - def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): - producer = self.get_pub() - future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, - timestamp_ms=timestamp_ms) - producer.close() + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + producer = self.get_conn() + try: + future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, + timestamp_ms=timestamp_ms) + finally: + producer.close() return future_record_metadata def __repr__(self): diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py index 738e63c76c730..e51328cdd63a9 100644 --- a/airflow/contrib/sensors/kafka_sensor.py +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -12,9 +12,11 @@ import logging -from src.kafka_hook import KafkaConsumerHook -from airflow.operators.sensors import BaseSensorOperator -from airflow.utils import apply_defaults +from werkzeug.utils import cached_property + +from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class KafkaSensor(BaseSensorOperator): @@ -22,6 +24,8 @@ class KafkaSensor(BaseSensorOperator): Consumes the Kafka message with the specific topic """ + templated_fields = ('topic',) + @apply_defaults def __init__(self, conn_id, topic, *args, **kwargs): """ @@ -34,9 +38,13 @@ def __init__(self, conn_id, topic, *args, **kwargs): the subscribed topic """ self.topic = topic - self.hook = KafkaConsumerHook(conn_id, topic) + self.conn_id = conn_id super(KafkaSensor, self).__init__(*args, **kwargs) + @cached_property + def hook(self): + return KafkaConsumerHook(self.conn_id, self.topic) + def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', From 4eaf3eef180d6154384fb2ae7bfe9dcc1f280d64 Mon Sep 17 00:00:00 2001 From: dan Date: Mon, 2 Mar 2020 15:47:10 -0500 Subject: [PATCH 122/162] Moved files to providers package. Externalized connection logic from init function --- airflow/contrib/hooks/kafka_consumer_hook.py | 71 ------------------- airflow/contrib/hooks/kafka_producer_hook.py | 58 --------------- airflow/contrib/sensors/kafka_sensor.py | 58 --------------- .../apache/kafka/hooks/kafka_consumer_hook.py | 21 ++---- .../apache/kafka/hooks/kafka_producer_hook.py | 31 ++------ .../apache/kafka/sensors/kafka_sensor.py | 33 ++++----- 6 files changed, 22 insertions(+), 250 deletions(-) delete mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py delete mode 100644 airflow/contrib/hooks/kafka_producer_hook.py delete mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py deleted file mode 100644 index 1143e2ccfb6a7..0000000000000 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ /dev/null @@ -1,71 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaConsumer - - -class KafkaConsumerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - # Disable auto commit as the hook will commit right - # after polling. - conf['enable_auto_commit'] = False - - self.server = f"""{host}:{port}""" - self.consumer = KafkaConsumer( - self.topic, - bootstrap_servers=self.server, **conf) - - return self.consumer - - def get_messages(self, timeout_ms=50): - """ - Get all the messages haven't been consumed, it doesn't - block by default, then commit the offset. - :return: - A list of messages - """ - consumer = self.get_conn() - try: - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms) - - consumer.commit() - finally: - consumer.close() - return messages - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.consumer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py deleted file mode 100644 index 9c1ebb158afe5..0000000000000 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaProducer - - -class KafkaProducerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaProducerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - conf['enable_auto_commit'] = False - self.server = f"""{host}:{port}""" - self.producer = KafkaProducer( - bootstrap_servers=self.server, **conf) - - return self.producer - - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - producer = self.get_conn() - try: - future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, - timestamp_ms=timestamp_ms) - finally: - producer.close() - return future_record_metadata - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.produer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py deleted file mode 100644 index e51328cdd63a9..0000000000000 --- a/airflow/contrib/sensors/kafka_sensor.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import logging - -from werkzeug.utils import cached_property - -from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook -from airflow.sensors.base_sensor_operator import BaseSensorOperator -from airflow.utils.decorators import apply_defaults - - -class KafkaSensor(BaseSensorOperator): - """ - Consumes the Kafka message with the specific topic - """ - - templated_fields = ('topic',) - - @apply_defaults - def __init__(self, conn_id, topic, *args, **kwargs): - """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - :param conn_id: - the kafka broker connection whom this sensor - subscripts against. - :param topic: - the subscribed topic - """ - self.topic = topic - self.conn_id = conn_id - super(KafkaSensor, self).__init__(*args, **kwargs) - - @cached_property - def hook(self): - return KafkaConsumerHook(self.conn_id, self.topic) - - def poke(self, context): - logging.info( - 'Poking topic: %s, using hook: %s', - self.topic, self.hook) - - messages = self.hook.get_messages() - - logging.info( - 'Got messages during poking: %s', messages) - - return messages or False diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 96472934ab63b..cbb2a4c963839 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -15,9 +15,7 @@ class KafkaConsumerHook(BaseHook): - """ - KafkaConsumerHook Class. - """ + DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 @@ -32,13 +30,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self) -> KafkaConsumer: - """ - A Kafka Consumer object. - - :return: - A Kafka Consumer object. - """ + def get_conn(self): if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -53,12 +45,10 @@ def get_conn(self) -> KafkaConsumer: ) return self.consumer - def get_messages(self, timeout_ms=5000) -> dict: + def get_messages(self, timeout_ms=5000): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. - - :param timeout_ms: :return: A list of messages """ @@ -72,10 +62,7 @@ def get_messages(self, timeout_ms=5000) -> dict: def __repr__(self): """ - A pretty version of the connection string. - - :return: - A pretty version of the connection string. + Pretty the hook with the connection info """ connected = self.consumer is not None return ' KafkaProducer: - """ - Returns a Kafka Producer - - :return: - A Kafka Producer object. - """ + def get_conn(self): if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -52,16 +42,6 @@ def get_conn(self) -> KafkaProducer: return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - """ - Sends a message on the specified topic and partition. Keyed messages will be sent in order. - - :param topic: - :param value: - :param key: - :param partition: - :param timestamp_ms: - :return: - """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, @@ -72,12 +52,9 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - A pretty version of the connection string. - - :return: - A pretty version of the connection string. + Pretty the hook with the connection info """ - connected = self.producer is not None + connected = self.produer is not None return '' % \ (connected, self.server, self.topic) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 46503c838d1b5..f7043a0294706 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -10,6 +10,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +import logging + from cached_property import cached_property from airflow.utils.decorators import apply_defaults from airflow.sensors.base_sensor_operator import BaseSensorOperator @@ -30,14 +32,13 @@ class KafkaSensor(BaseSensorOperator): @apply_defaults def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs): """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. :param topic: - :param host: - :param port: - :param args: - :param kwargs: + the subscribed topic """ self.topic = topic self.host = host @@ -46,24 +47,18 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): - """ - Returns a Kafka Consumer Hook - """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): - """ - Checks to see if messages exist on this topic/partition. - - :param context: - :return: - """ - self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) messages = self.hook.get_messages() - if messages: - self.log.info('Got messages during poking: %s', str(messages)) + if messages is not {}: + logging.info( + 'Got messages during poking: %s', messages) return messages else: return False From b2928a58b18fb93d301351ffc4637b7cd4a5ce83 Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 11 Mar 2020 09:06:55 -0400 Subject: [PATCH 123/162] Fixing Travis CI Linting errors --- .../apache/kafka/hooks/kafka_consumer_hook.py | 2 +- .../apache/kafka/hooks/kafka_producer_hook.py | 2 +- .../apache/kafka/sensors/kafka_sensor.py | 24 ++++++++++++------- 3 files changed, 17 insertions(+), 11 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index cbb2a4c963839..968514398a20b 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -10,8 +10,8 @@ # See the License for the specific language governing permissions and # limitations under the License. -from airflow.hooks import base_hook as BaseHook from kafka import KafkaConsumer +from airflow.hooks.base_hook import BaseHook class KafkaConsumerHook(BaseHook): diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 4bc60e2a610fd..7d74110012529 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -10,8 +10,8 @@ # See the License for the specific language governing permissions and # limitations under the License. -from airflow.hooks import base_hook as BaseHook from kafka import KafkaProducer +from airflow.hooks.base_hook import BaseHook class KafkaProducerHook(BaseHook): diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index f7043a0294706..c5096d8734fcd 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -1,14 +1,20 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. import logging From 35f0726f5f96baa24d98c404741b1cb8472e9613 Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 11 Mar 2020 09:06:55 -0400 Subject: [PATCH 124/162] Fixing Travis CI Linting errors --- .../apache/kafka/hooks/kafka_consumer_hook.py | 25 ++++++++----- .../apache/kafka/hooks/kafka_producer_hook.py | 35 +++++++++++-------- 2 files changed, 37 insertions(+), 23 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 968514398a20b..bb4895fab25fb 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -1,14 +1,21 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + from kafka import KafkaConsumer from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 7d74110012529..e7f70f2ef1aec 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -1,14 +1,20 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. from kafka import KafkaProducer from airflow.hooks.base_hook import BaseHook @@ -21,7 +27,8 @@ class KafkaProducerHook(BaseHook): def __init__(self, conn_id, topic): super(KafkaProducerHook, self).__init__(None) - self.conn = None + self.conn_id = conn_id + self._conn = None self.server = None self.consumer = None self.producer = None @@ -29,10 +36,10 @@ def __init__(self, conn_id, topic): def get_conn(self): if not self._conn: - conn = self.get_connection(self.conn_id) - service_options = conn.extra_dejson - host = conn.host or self.DEFAULT_HOST - port = conn.port or self.DEFAULT_PORT + _conn = self.get_connection(self.conn_id) + service_options = _conn.extra_dejson + host = _conn.host or self.DEFAULT_HOST + port = _conn.port or self.DEFAULT_PORT self.server = f"""{host}:{port}""" self.consumer = KafkaProducer( From 81d086d821e69ebefb3700dcc1ba9a579d80383d Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:16:08 -0400 Subject: [PATCH 125/162] produer --> producer --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index e7f70f2ef1aec..0d5f487c3ff85 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -61,7 +61,7 @@ def __repr__(self): """ Pretty the hook with the connection info """ - connected = self.produer is not None + connected = self.producer is not None return '' % \ (connected, self.server, self.topic) From 8f33a4e9bee1180e4448c4dc8a0648c317073806 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 09:36:16 -0400 Subject: [PATCH 126/162] Added pylint mandated doc strings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 2 ++ .../apache/kafka/hooks/kafka_producer_hook.py | 11 +++++++++++ .../providers/apache/kafka/sensors/kafka_sensor.py | 7 ++++++- 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index bb4895fab25fb..a9abdcef3c148 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -22,7 +22,9 @@ class KafkaConsumerHook(BaseHook): + """ + """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 0d5f487c3ff85..3faf5c379d33e 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -21,7 +21,9 @@ class KafkaProducerHook(BaseHook): + """ + """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 @@ -49,6 +51,15 @@ def get_conn(self): return self.producer def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + """ + + :param topic: + :param value: + :param key: + :param partition: + :param timestamp_ms: + :return: + """ producer = self.get_conn() try: future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index c5096d8734fcd..e8119da6fc1e1 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -53,12 +53,17 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): + """ + + :return: + KafkaConsumerHook + """ return KafkaConsumerHook(self.topic, self.host, self.port) def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', - self.topic, self.hook) + str(self.topic), str(self.hook)) messages = self.hook.get_messages() From 0f5c5c245ac88cf61896aa55d1a5c165b89f707c Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 5 May 2020 10:06:05 -0400 Subject: [PATCH 127/162] Fixing more pylint issues --- airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py | 2 +- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index a9abdcef3c148..9270c6bbf25c5 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -23,7 +23,7 @@ class KafkaConsumerHook(BaseHook): """ - + KafkaConsumerHook Class. """ DEFAULT_HOST = 'kafka1' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 3faf5c379d33e..845686fab9979 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -22,7 +22,7 @@ class KafkaProducerHook(BaseHook): """ - + KafkaProducerHook Class. """ DEFAULT_HOST = 'localhost' DEFAULT_PORT = 9092 diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index e8119da6fc1e1..48ab157e94078 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -67,9 +67,9 @@ def poke(self, context): messages = self.hook.get_messages() - if messages is not {}: + if not messages: logging.info( - 'Got messages during poking: %s', messages) + 'Got messages during poking: %s', str(messages)) return messages else: return False From de9e68f3eaa8787ffde3620fac133407d418f8d5 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 128/162] Tried to fix docstrings --- .../apache/kafka/hooks/kafka_consumer_hook.py | 17 +++++++++++--- .../apache/kafka/hooks/kafka_producer_hook.py | 17 +++++++++++--- .../apache/kafka/sensors/kafka_sensor.py | 22 +++++++++++++------ 3 files changed, 43 insertions(+), 13 deletions(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 9270c6bbf25c5..6b58bd3091d1a 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -39,7 +39,13 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, kafka_conn_id='k self.host = host self.port = port - def get_conn(self): + def get_conn(self) -> KafkaConsumer: + """ + A Kafka Consumer object. + + :return: + A Kafka Consumer object. + """ if not self._conn: conn = self.get_connection(self.conn_id) service_options = conn.extra_dejson @@ -54,10 +60,12 @@ def get_conn(self): ) return self.consumer - def get_messages(self, timeout_ms=5000): + def get_messages(self, timeout_ms=5000) -> dict: """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. + + :param timeout_ms: :return: A list of messages """ @@ -71,7 +79,10 @@ def get_messages(self, timeout_ms=5000): def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.consumer is not None return ' KafkaProducer: + """ + Returns a Kafka Producer + + :return: + A Kafka Producer object. + """ if not self._conn: _conn = self.get_connection(self.conn_id) service_options = _conn.extra_dejson @@ -50,8 +57,9 @@ def get_conn(self): ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: """ + Sends a message on the specified topic and partition. Keyed messages will be sent in order. :param topic: :param value: @@ -70,7 +78,10 @@ def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms def __repr__(self): """ - Pretty the hook with the connection info + A pretty version of the connection string. + + :return: + A pretty version of the connection string. """ connected = self.producer is not None return ' Date: Fri, 8 May 2020 12:29:02 -0400 Subject: [PATCH 129/162] Tried to fix docstrings --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index ce50246e7f000..d67ce1eee97e7 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -57,7 +57,7 @@ def get_conn(self) -> KafkaProducer: ) return self.producer - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None) -> FutureRecordMetadata: + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): """ Sends a message on the specified topic and partition. Keyed messages will be sent in order. From 5c6ae5bb257b0c14f8c2f675a9b8ed751fca642e Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 9 May 2020 23:41:58 +0100 Subject: [PATCH 130/162] Apply suggestions from code review --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 158eae23270d8..636a29035a5d3 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -55,10 +55,7 @@ def __init__(self, topic, host=DEFAULT_HOST, port=DEFAULT_PORT, *args, **kwargs) @cached_property def hook(self): """ - Returns a Kafka Consumer Hook - - :return: - KafkaConsumerHook + Returns a Kafka Consumer Hook """ return KafkaConsumerHook(self.topic, self.host, self.port) From baf46c2d1b5a8d8dd0752760253086868ba20877 Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:28 -0400 Subject: [PATCH 131/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 636a29035a5d3..58d3e203ccc44 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -73,8 +73,7 @@ def poke(self, context): messages = self.hook.get_messages() if messages: - logging.info( - 'Got messages during poking: %s', str(messages)) + self.log.info('Got messages during poking: %s', str(messages)) return messages else: return False From fa27127380c7ad664573d2ab61bd3b702fc52f8b Mon Sep 17 00:00:00 2001 From: Dan-o Date: Tue, 12 May 2020 08:51:46 -0400 Subject: [PATCH 132/162] Update airflow/providers/apache/kafka/sensors/kafka_sensor.py Co-authored-by: Kaxil Naik --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 58d3e203ccc44..16c1c5ed4405d 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -66,9 +66,7 @@ def poke(self, context): :param context: :return: """ - logging.info( - 'Poking topic: %s, using hook: %s', - str(self.topic), str(self.hook)) + self.log.info('Poking topic: %s, using hook: %s', str(self.topic), str(self.hook)) messages = self.hook.get_messages() From c95686c5e879bd5dc055af73486540125de9a087 Mon Sep 17 00:00:00 2001 From: dan Date: Sat, 23 May 2020 14:46:59 -0400 Subject: [PATCH 133/162] Removed unused imports --- airflow/providers/apache/kafka/sensors/kafka_sensor.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 16c1c5ed4405d..481c69a321557 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -16,8 +16,6 @@ # specific language governing permissions and limitations # under the License. -import logging - from cached_property import cached_property from airflow.utils.decorators import apply_defaults from airflow.sensors.base_sensor_operator import BaseSensorOperator From c0463b2c884269adc86dbc6068d0ceef0628701b Mon Sep 17 00:00:00 2001 From: dan Date: Sat, 23 May 2020 14:46:59 -0400 Subject: [PATCH 134/162] Removed unused imports --- setup.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/setup.py b/setup.py index eaf744725f366..19febb59df53a 100644 --- a/setup.py +++ b/setup.py @@ -565,10 +565,13 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version "apache.druid": druid, "apache.hdfs": hdfs, "apache.hive": hive, +<<<<<<< HEAD <<<<<<< HEAD "apache.kylin": kylin, ======= >>>>>>> 9ad001e9a... Removed unused imports +======= +>>>>>>> 8ed04aed7... Removed unused imports "apache.kafka": kafka, "apache.pinot": pinot, "apache.webhdfs": webhdfs, From 8f9a560c45afb290a99d84a2875aae66f7a365da Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 16:12:11 -0400 Subject: [PATCH 135/162] added the python 3.8 requirements --- requirements/requirements-python3.6.txt | 4 ++-- requirements/requirements-python3.7.txt | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index 3c7ee9b2b3896..0c4ee0c0ebdb2 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -56,7 +56,7 @@ azure-batch==9.0.0 azure-common==1.1.25 azure-cosmos==3.2.0 azure-datalake-store==0.0.48 -azure-kusto-data==0.0.45 +azure-kusto-data==0.1.0 azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 @@ -163,7 +163,7 @@ google-cloud-spanner==1.17.1 google-cloud-speech==1.3.2 google-cloud-storage==1.29.0 google-cloud-tasks==1.5.0 -google-cloud-texttospeech==1.0.1 +google-cloud-texttospeech==2.0.0 google-cloud-translate==2.0.1 google-cloud-videointelligence==1.15.0 google-cloud-vision==1.0.0 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index ae515b109b793..749e26985d6c0 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -56,7 +56,7 @@ azure-batch==9.0.0 azure-common==1.1.25 azure-cosmos==3.2.0 azure-datalake-store==0.0.48 -azure-kusto-data==0.0.45 +azure-kusto-data==0.1.0 azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 @@ -162,7 +162,7 @@ google-cloud-spanner==1.17.1 google-cloud-speech==1.3.2 google-cloud-storage==1.29.0 google-cloud-tasks==1.5.0 -google-cloud-texttospeech==1.0.1 +google-cloud-texttospeech==2.0.0 google-cloud-translate==2.0.1 google-cloud-videointelligence==1.15.0 google-cloud-vision==1.0.0 From edb2a4eecdc6ca88bd3edb4f2fd6a12a589587bb Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 17:38:27 -0400 Subject: [PATCH 136/162] updating dependencies files --- airflow/providers/docker/example_dags/example_docker_swarm.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/airflow/providers/docker/example_dags/example_docker_swarm.py b/airflow/providers/docker/example_dags/example_docker_swarm.py index 3930482ec1fa1..a4f50917be5bd 100644 --- a/airflow/providers/docker/example_dags/example_docker_swarm.py +++ b/airflow/providers/docker/example_dags/example_docker_swarm.py @@ -19,10 +19,8 @@ from datetime import timedelta from airflow import DAG -from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor from airflow.providers.docker.operators.docker_swarm import DockerSwarmOperator from airflow.utils.dates import days_ago -from airflow.utils.timezone import datetime default_args = { 'owner': 'airflow', From 35b548d86290c2cc73d9082a6121dc4ef337bc11 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 18:24:31 -0400 Subject: [PATCH 137/162] updating requirements again --- requirements/requirements-python3.6.txt | 7 +++++-- requirements/requirements-python3.7.txt | 9 ++++++--- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index 0c4ee0c0ebdb2..1d2beb53a74ae 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -56,7 +56,7 @@ azure-batch==9.0.0 azure-common==1.1.25 azure-cosmos==3.2.0 azure-datalake-store==0.0.48 -azure-kusto-data==0.1.0 +azure-kusto-data==0.0.45 azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 @@ -101,6 +101,7 @@ cryptography==2.9.2 curlify==2.2.1 cx-Oracle==8.0.0 dask==2.20.0 +dataclasses==0.7 datadog==0.38.0 decorator==4.4.2 defusedxml==0.6.0 @@ -163,7 +164,7 @@ google-cloud-spanner==1.17.1 google-cloud-speech==1.3.2 google-cloud-storage==1.29.0 google-cloud-tasks==1.5.0 -google-cloud-texttospeech==2.0.0 +google-cloud-texttospeech==1.0.1 google-cloud-translate==2.0.1 google-cloud-videointelligence==1.15.0 google-cloud-vision==1.0.0 @@ -213,6 +214,7 @@ kubernetes==11.0.0 kylinpy==2.8.1 lazy-object-proxy==1.5.0 ldap3==2.7 +libcst==0.3.6 lockfile==0.12.2 marshmallow-enum==1.5.1 marshmallow-oneofschema==2.0.1 @@ -266,6 +268,7 @@ presto-python-client==0.7.0 prison==0.1.3 prometheus-client==0.8.0 prompt-toolkit==3.0.5 +proto-plus==0.4.0 protobuf==3.12.2 psutil==5.7.0 psycopg2-binary==2.8.5 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index 749e26985d6c0..ced25f19a306a 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -56,7 +56,7 @@ azure-batch==9.0.0 azure-common==1.1.25 azure-cosmos==3.2.0 azure-datalake-store==0.0.48 -azure-kusto-data==0.1.0 +azure-kusto-data==0.0.45 azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 @@ -162,7 +162,7 @@ google-cloud-spanner==1.17.1 google-cloud-speech==1.3.2 google-cloud-storage==1.29.0 google-cloud-tasks==1.5.0 -google-cloud-texttospeech==2.0.0 +google-cloud-texttospeech==1.0.1 google-cloud-translate==2.0.1 google-cloud-videointelligence==1.15.0 google-cloud-vision==1.0.0 @@ -210,6 +210,7 @@ kubernetes==11.0.0 kylinpy==2.8.1 lazy-object-proxy==1.5.0 ldap3==2.7 +libcst==0.3.6 lockfile==0.12.2 marshmallow-enum==1.5.1 marshmallow-oneofschema==2.0.1 @@ -262,6 +263,7 @@ presto-python-client==0.7.0 prison==0.1.3 prometheus-client==0.8.0 prompt-toolkit==3.0.5 +proto-plus==0.4.0 protobuf==3.12.2 psutil==5.7.0 psycopg2-binary==2.8.5 @@ -373,6 +375,7 @@ tqdm==4.47.0 traitlets==4.3.3 typed-ast==1.4.1 typing-extensions==3.7.4.2 +typing-inspect==0.6.0 tzlocal==1.5.1 unicodecsv==0.14.1 uritemplate==3.0.1 @@ -383,7 +386,7 @@ virtualenv==20.0.26 watchtower==0.7.3 wcwidth==0.2.5 websocket-client==0.57.0 -wrapt==1.12.1 +wrapt==1.11.2 xmltodict==0.12.0 yamllint==1.23.0 yandexcloud==0.43.0 From af69ccc835cb3db262b7f7d657ea3a84d428e689 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 21:04:17 -0400 Subject: [PATCH 138/162] Fixed yaml linting issues --- scripts/ci/docker-compose/local.yml | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml index cda6ce89aaa84..ac3eeb9ba6401 100755 --- a/scripts/ci/docker-compose/local.yml +++ b/scripts/ci/docker-compose/local.yml @@ -14,18 +14,16 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +--- version: "2.2" services: airflow: - stdin_open: true # docker run -i - tty: true # docker run -t # We need to mount files an directories individually because some files # such apache_airflow.egg-info should not be mounted from host # we only mount those files that it makes sense to edit while developing # or those that might be useful to see in the host as output of the # tests (such as logs) volumes: - # START automatically generated volumes from LOCAL_MOUNTS in _local_mounts.sh - ../../../.bash_aliases:/root/.bash_aliases:cached - ../../../.bash_history:/root/.bash_history:cached - ../../../.coveragerc:/opt/airflow/.coveragerc:cached From 3d44718117fa5e81488b0f278044eadb9d9b5722 Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 17 Jun 2020 09:53:14 -0400 Subject: [PATCH 139/162] Modified file perms --- scripts/ci/docker-compose/local.yml | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 scripts/ci/docker-compose/local.yml diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml old mode 100755 new mode 100644 From 108508451e0a944ff82fd87cbaac9820f70a8507 Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 17 Jun 2020 09:53:14 -0400 Subject: [PATCH 140/162] Modified file perms --- scripts/ci/docker-compose/local.yml | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml index ac3eeb9ba6401..fa1354bf14046 100644 --- a/scripts/ci/docker-compose/local.yml +++ b/scripts/ci/docker-compose/local.yml @@ -24,6 +24,7 @@ services: # or those that might be useful to see in the host as output of the # tests (such as logs) volumes: + # START automatically generated volumes from LOCAL_MOUNTS in _local_mounts.sh - ../../../.bash_aliases:/root/.bash_aliases:cached - ../../../.bash_history:/root/.bash_history:cached - ../../../.coveragerc:/opt/airflow/.coveragerc:cached @@ -53,11 +54,7 @@ services: - ../../../tests:/opt/airflow/tests:cached - ../../../kubernetes_tests:/opt/airflow/kubernetes_tests:cached - ../../../tmp:/tmp:cached - - ../../../metastore_browser:/opt/airflow/metastore_browser:cached # END automatically generated volumes from LOCAL_MOUNTS in _local_mounts.sh - - ../../../tmp:/opt/airflow/tmp:cached - # END automatically generated volumes from LOCAL_MOUNTS in - # _local_mounts.sh environment: - HOST_USER_ID - HOST_GROUP_ID From a6d55a46016992c43dff8170fccbd2811791b1ce Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 17 Jun 2020 12:12:17 -0400 Subject: [PATCH 141/162] Updated requirements files again --- .../providers/docker/example_dags/example_docker_swarm.py | 1 + docs/operators-and-hooks-ref.rst | 1 + requirements/requirements-python3.6.txt | 4 +--- requirements/requirements-python3.7.txt | 5 +---- requirements/requirements-python3.8.txt | 1 + 5 files changed, 5 insertions(+), 7 deletions(-) diff --git a/airflow/providers/docker/example_dags/example_docker_swarm.py b/airflow/providers/docker/example_dags/example_docker_swarm.py index a4f50917be5bd..1bcf37bc93423 100644 --- a/airflow/providers/docker/example_dags/example_docker_swarm.py +++ b/airflow/providers/docker/example_dags/example_docker_swarm.py @@ -22,6 +22,7 @@ from airflow.providers.docker.operators.docker_swarm import DockerSwarmOperator from airflow.utils.dates import days_ago + default_args = { 'owner': 'airflow', 'depends_on_past': False, diff --git a/docs/operators-and-hooks-ref.rst b/docs/operators-and-hooks-ref.rst index 6ff38f4aa5e77..4066e8aefee92 100644 --- a/docs/operators-and-hooks-ref.rst +++ b/docs/operators-and-hooks-ref.rst @@ -146,6 +146,7 @@ Foundation. - - :mod:`airflow.providers.apache.kafka.hooks.kafka_producer_hook`, :mod:`airflow.providers.apache.kafka.hooks.kafka_consumer_hook` + - - :mod:`airflow.providers.apache.kafka.sensors.kafka_sensor` * - `Apache Hive `__ diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index 1d2beb53a74ae..5a8bf4a6baf1e 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -214,7 +214,6 @@ kubernetes==11.0.0 kylinpy==2.8.1 lazy-object-proxy==1.5.0 ldap3==2.7 -libcst==0.3.6 lockfile==0.12.2 marshmallow-enum==1.5.1 marshmallow-oneofschema==2.0.1 @@ -268,7 +267,6 @@ presto-python-client==0.7.0 prison==0.1.3 prometheus-client==0.8.0 prompt-toolkit==3.0.5 -proto-plus==0.4.0 protobuf==3.12.2 psutil==5.7.0 psycopg2-binary==2.8.5 @@ -393,7 +391,7 @@ virtualenv==20.0.26 watchtower==0.7.3 wcwidth==0.2.5 websocket-client==0.57.0 -wrapt==1.12.1 +wrapt==1.11.2 xmltodict==0.12.0 yamllint==1.23.0 yandexcloud==0.43.0 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index ced25f19a306a..ae515b109b793 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -210,7 +210,6 @@ kubernetes==11.0.0 kylinpy==2.8.1 lazy-object-proxy==1.5.0 ldap3==2.7 -libcst==0.3.6 lockfile==0.12.2 marshmallow-enum==1.5.1 marshmallow-oneofschema==2.0.1 @@ -263,7 +262,6 @@ presto-python-client==0.7.0 prison==0.1.3 prometheus-client==0.8.0 prompt-toolkit==3.0.5 -proto-plus==0.4.0 protobuf==3.12.2 psutil==5.7.0 psycopg2-binary==2.8.5 @@ -375,7 +373,6 @@ tqdm==4.47.0 traitlets==4.3.3 typed-ast==1.4.1 typing-extensions==3.7.4.2 -typing-inspect==0.6.0 tzlocal==1.5.1 unicodecsv==0.14.1 uritemplate==3.0.1 @@ -386,7 +383,7 @@ virtualenv==20.0.26 watchtower==0.7.3 wcwidth==0.2.5 websocket-client==0.57.0 -wrapt==1.11.2 +wrapt==1.12.1 xmltodict==0.12.0 yamllint==1.23.0 yandexcloud==0.43.0 diff --git a/requirements/requirements-python3.8.txt b/requirements/requirements-python3.8.txt index ccde845c95c99..3fcb187cd2e04 100644 --- a/requirements/requirements-python3.8.txt +++ b/requirements/requirements-python3.8.txt @@ -181,6 +181,7 @@ humanize==2.5.0 hvac==0.10.4 identify==1.4.23 idna==2.10 +ijson==2.6.1 imagesize==1.2.0 importlib-metadata==1.7.0 inflection==0.5.0 From b7558695637f1216ed8bc3ee85d017f1d0339170 Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 17 Jun 2020 12:29:04 -0400 Subject: [PATCH 142/162] Fixed indentation issue --- docs/operators-and-hooks-ref.rst | 2 +- scripts/ci/docker-compose/local.yml | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/operators-and-hooks-ref.rst b/docs/operators-and-hooks-ref.rst index 4066e8aefee92..eed212044df57 100644 --- a/docs/operators-and-hooks-ref.rst +++ b/docs/operators-and-hooks-ref.rst @@ -146,7 +146,7 @@ Foundation. - - :mod:`airflow.providers.apache.kafka.hooks.kafka_producer_hook`, :mod:`airflow.providers.apache.kafka.hooks.kafka_consumer_hook` - - + - - :mod:`airflow.providers.apache.kafka.sensors.kafka_sensor` * - `Apache Hive `__ diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml index fa1354bf14046..66339a6b40462 100644 --- a/scripts/ci/docker-compose/local.yml +++ b/scripts/ci/docker-compose/local.yml @@ -18,6 +18,8 @@ version: "2.2" services: airflow: + stdin_open: true # docker run -i + tty: true # docker run -t # We need to mount files an directories individually because some files # such apache_airflow.egg-info should not be mounted from host # we only mount those files that it makes sense to edit while developing From 8262de2c7cda2ac23f1b41e2b1274d7cff0d453e Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 17 Jun 2020 12:52:17 -0400 Subject: [PATCH 143/162] Removed trailing whitespace --- docs/operators-and-hooks-ref.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/operators-and-hooks-ref.rst b/docs/operators-and-hooks-ref.rst index eed212044df57..9030fa5d9be63 100644 --- a/docs/operators-and-hooks-ref.rst +++ b/docs/operators-and-hooks-ref.rst @@ -146,7 +146,7 @@ Foundation. - - :mod:`airflow.providers.apache.kafka.hooks.kafka_producer_hook`, :mod:`airflow.providers.apache.kafka.hooks.kafka_consumer_hook` - - + - - :mod:`airflow.providers.apache.kafka.sensors.kafka_sensor` * - `Apache Hive `__ From 033cd6c296ee70a1838f3233460e12c8e953242d Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 12 Feb 2020 13:51:03 -0500 Subject: [PATCH 144/162] Add KafkaConsumerHook, KafkaProduerHook and KafkaSensor MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Dan Ferguson dferguson992@gmail.com Co-authored-by: Hanan Shteingart Co-authored-by: YuanfΞi Zhu --- .../kafka/example_dags/example_kafka_dag.py | 20 ++++++++++--------- .../apache/kafka/hooks/kafka_consumer_hook.py | 1 + .../apache/kafka/sensors/kafka_sensor.py | 4 ++-- docs/autoapi_templates/index.rst | 4 ++++ setup.py | 8 +------- 5 files changed, 19 insertions(+), 18 deletions(-) diff --git a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py index 0d718ba2c80cf..d3515f9887439 100644 --- a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py +++ b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py @@ -16,28 +16,30 @@ # specific language governing permissions and limitations # under the License. -from airflow import DAG from datetime import timedelta -from airflow.utils.timezone import datetime +from airflow import DAG from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor +from airflow.utils.timezone import datetime -DAG_ID = "example_kafka_dag" -dag_start_date = datetime(2015, 6, 1, hour=20, tzinfo=None) default_args = { 'owner': '@Ferg_In', 'depends_on_past': False, - 'start_date': dag_start_date, - 'email': ['dferguson992@gmail.com'], 'provide_context': True, 'retries': 3, 'retry_delay': timedelta(minutes=5), } -with DAG(dag_id=DAG_ID, default_args=default_args, schedule_interval=None, - max_active_runs=1, concurrency=4, catchup=False) as dag: - +with DAG( + dag_id="example_kafka_dag", + default_args=default_args, + schedule_interval=None, + start_date=datetime(2015, 6, 1, hour=20, tzinfo=None), + max_active_runs=1, + concurrency=4, + catchup=False +) as dag: sensor = KafkaSensor( task_id='trigger', topic='', diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 6b58bd3091d1a..239aa5e025bd4 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -45,6 +45,7 @@ def get_conn(self) -> KafkaConsumer: :return: A Kafka Consumer object. + """ if not self._conn: conn = self.get_connection(self.conn_id) diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 481c69a321557..997294cc4ed2b 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -17,9 +17,9 @@ # under the License. from cached_property import cached_property -from airflow.utils.decorators import apply_defaults -from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class KafkaSensor(BaseSensorOperator): diff --git a/docs/autoapi_templates/index.rst b/docs/autoapi_templates/index.rst index cb4eb74c116a5..e3bb10a1e515f 100644 --- a/docs/autoapi_templates/index.rst +++ b/docs/autoapi_templates/index.rst @@ -86,6 +86,10 @@ All operators are in the following packages: airflow/providers/apache/hive/transfers/index + airflow/providers/apache/kafka/sensors/index + + airflow/providers/apache/kafka/hooks/index + airflow/providers/apache/kylin/operators/index airflow/providers/apache/kafka/sensors/index diff --git a/setup.py b/setup.py index 19febb59df53a..5aad5e84de625 100644 --- a/setup.py +++ b/setup.py @@ -565,14 +565,8 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version "apache.druid": druid, "apache.hdfs": hdfs, "apache.hive": hive, -<<<<<<< HEAD -<<<<<<< HEAD - "apache.kylin": kylin, -======= ->>>>>>> 9ad001e9a... Removed unused imports -======= ->>>>>>> 8ed04aed7... Removed unused imports "apache.kafka": kafka, + "apache.kylin": kylin, "apache.pinot": pinot, "apache.webhdfs": webhdfs, 'async': async_packages, From d4af358b258382ca392b5d2d899de9ac27fdc715 Mon Sep 17 00:00:00 2001 From: dan Date: Mon, 24 Aug 2020 22:49:59 -0400 Subject: [PATCH 145/162] Attempted to rebase? --- airflow/providers/apache/kafka/example_dags/example_kafka_dag.py | 1 - airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py | 1 + airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 1 + airflow/providers/apache/kafka/sensors/kafka_sensor.py | 1 + airflow/providers/docker/example_dags/example_docker_swarm.py | 1 - 5 files changed, 3 insertions(+), 2 deletions(-) diff --git a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py index d3515f9887439..0ba9bbc8c7cb1 100644 --- a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py +++ b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py @@ -22,7 +22,6 @@ from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor from airflow.utils.timezone import datetime - default_args = { 'owner': '@Ferg_In', 'depends_on_past': False, diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py index 239aa5e025bd4..589c5de953891 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py @@ -18,6 +18,7 @@ from kafka import KafkaConsumer + from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index d67ce1eee97e7..9919505b4a24d 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -18,6 +18,7 @@ from kafka import KafkaProducer from kafka.producer.future import FutureRecordMetadata + from airflow.hooks.base_hook import BaseHook diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py index 997294cc4ed2b..842643c60ba40 100644 --- a/airflow/providers/apache/kafka/sensors/kafka_sensor.py +++ b/airflow/providers/apache/kafka/sensors/kafka_sensor.py @@ -17,6 +17,7 @@ # under the License. from cached_property import cached_property + from airflow.providers.apache.kafka.hooks.kafka_consumer_hook import KafkaConsumerHook from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/providers/docker/example_dags/example_docker_swarm.py b/airflow/providers/docker/example_dags/example_docker_swarm.py index 1bcf37bc93423..a4f50917be5bd 100644 --- a/airflow/providers/docker/example_dags/example_docker_swarm.py +++ b/airflow/providers/docker/example_dags/example_docker_swarm.py @@ -22,7 +22,6 @@ from airflow.providers.docker.operators.docker_swarm import DockerSwarmOperator from airflow.utils.dates import days_ago - default_args = { 'owner': 'airflow', 'depends_on_past': False, From 5dbd7366ac7649483e72840e7c22fee5d927090e Mon Sep 17 00:00:00 2001 From: dan Date: Mon, 24 Aug 2020 23:15:35 -0400 Subject: [PATCH 146/162] ?? --- CONTRIBUTING.rst | 18 +++++++++--------- INSTALL | 16 ++++++++-------- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/CONTRIBUTING.rst b/CONTRIBUTING.rst index 3fc8100d21832..4ae7377f9d10f 100644 --- a/CONTRIBUTING.rst +++ b/CONTRIBUTING.rst @@ -432,15 +432,15 @@ This is the full list of those extras: .. START EXTRAS HERE -all_dbs, amazon, apache.atlas, apache_beam, apache.cassandra, apache.druid, apache.hdfs, -apache.hive, apache.kafka, apache.pinot, apache.webhdfs, async, atlas, aws, azure, cassandra, -celery, cgroups, cloudant, cncf.kubernetes, dask, databricks, datadog, devel, devel_hadoop, doc, -docker, druid, elasticsearch, exasol, facebook, gcp, gcp_api, github_enterprise, google, -google_auth, grpc, hashicorp, hdfs, hive, jdbc, jira, kerberos, kubernetes, ldap, microsoft.azure, -microsoft.mssql, microsoft.winrm, mongo, mssql, mysql, odbc, oracle, pagerduty, papermill, password, -pinot, postgres, presto, qds, rabbitmq, redis, salesforce, samba, segment, sendgrid, sentry, -singularity, slack, snowflake, spark, ssh, statsd, tableau, vertica, virtualenv, webhdfs, winrm, -yandexcloud, all, devel_ci +all_dbs, amazon, apache.atlas, apache.beam, apache.cassandra, apache.druid, apache.hdfs, +apache.hive, apache.kafka, apache.kylin, apache.pinot, apache.webhdfs, async, atlas, aws, azure, +cassandra, celery, cgroups, cloudant, cncf.kubernetes, dask, databricks, datadog, devel, +devel_hadoop, doc, docker, druid, elasticsearch, exasol, facebook, gcp, gcp_api, github_enterprise, +google, google_auth, grpc, hashicorp, hdfs, hive, jdbc, jira, kerberos, kubernetes, ldap, +microsoft.azure, microsoft.mssql, microsoft.winrm, mongo, mssql, mysql, odbc, oracle, pagerduty, +papermill, password, pinot, postgres, presto, qds, rabbitmq, redis, salesforce, samba, segment, +sendgrid, sentry, singularity, slack, snowflake, spark, ssh, statsd, tableau, vertica, virtualenv, +webhdfs, winrm, yandexcloud, all, devel_ci .. END EXTRAS HERE diff --git a/INSTALL b/INSTALL index 8c2cc919ccfbb..b101822898d15 100644 --- a/INSTALL +++ b/INSTALL @@ -46,14 +46,14 @@ pip install . \ # START EXTRAS HERE all_dbs, amazon, apache.atlas, apache.beam, apache.cassandra, apache.druid, apache.hdfs, -apache.hive, apache.kylin, apache.pinot, apache.webhdfs, async, atlas, aws, azure, cassandra, -celery, cgroups, cloudant, cncf.kubernetes, dask, databricks, datadog, devel, devel_hadoop, doc, -docker, druid, elasticsearch, exasol, facebook, gcp, gcp_api, github_enterprise, google, -google_auth, grpc, hashicorp, hdfs, hive, jdbc, jira, kerberos, kubernetes, ldap, microsoft.azure, -microsoft.mssql, microsoft.winrm, mongo, mssql, mysql, odbc, oracle, pagerduty, papermill, password, -pinot, postgres, presto, qds, rabbitmq, redis, salesforce, samba, segment, sendgrid, sentry, -singularity, slack, snowflake, spark, ssh, statsd, tableau, vertica, virtualenv, webhdfs, winrm, -yandexcloud, all, devel_ci +apache.hive, apache.kafka, apache.kylin, apache.pinot, apache.webhdfs, async, atlas, aws, azure, +cassandra, celery, cgroups, cloudant, cncf.kubernetes, dask, databricks, datadog, devel, +devel_hadoop, doc, docker, druid, elasticsearch, exasol, facebook, gcp, gcp_api, github_enterprise, +google, google_auth, grpc, hashicorp, hdfs, hive, jdbc, jira, kerberos, kubernetes, ldap, +microsoft.azure, microsoft.mssql, microsoft.winrm, mongo, mssql, mysql, odbc, oracle, pagerduty, +papermill, password, pinot, postgres, presto, qds, rabbitmq, redis, salesforce, samba, segment, +sendgrid, sentry, singularity, slack, snowflake, spark, ssh, statsd, tableau, vertica, virtualenv, +webhdfs, winrm, yandexcloud, all, devel_ci # END EXTRAS HERE From f5ec5719f60aaeb4dfe2647573e40afba25793e1 Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 12 Feb 2020 13:51:03 -0500 Subject: [PATCH 147/162] Add KafkaConsumerHook, KafkaProduerHook and KafkaSensor MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Dan Ferguson dferguson992@gmail.com Co-authored-by: Hanan Shteingart Co-authored-by: YuanfΞi Zhu --- airflow/contrib/hooks/kafka_consumer_hook.py | 69 ++++++++++++++++++++ airflow/contrib/hooks/kafka_producer_hook.py | 56 ++++++++++++++++ airflow/contrib/sensors/kafka_sensor.py | 50 ++++++++++++++ 3 files changed, 175 insertions(+) create mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py create mode 100644 airflow/contrib/hooks/kafka_producer_hook.py create mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py new file mode 100644 index 0000000000000..335c267b7837f --- /dev/null +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -0,0 +1,69 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaConsumer + + +class KafkaConsumerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + # Disable auto commit as the hook will commit right + # after polling. + conf['enable_auto_commit'] = False + + self.server = '{host}:{port}'.format(**locals()) + self.consumer = KafkaConsumer( + self.topic, + bootstrap_servers=self.server, **conf) + + return self.consumer + + def get_messages(self): + """ + Get all the messages haven't been consumed, it doesn't + block by default, then commit the offset. + :return: + A list of messages + """ + consumer = self.get_conn() + + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms=50) + + consumer.commit() + return messages + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.consumer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py new file mode 100644 index 0000000000000..ffa5d4693e9c2 --- /dev/null +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -0,0 +1,56 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from airflow.hooks import base_hook as BaseHook +from kafka import KafkaProducer + + +class KafkaProducerHook(BaseHook): + + default_host = 'localhost' + default_port = 9092 + + def __init__(self, conn_id, topic): + super(KafkaConsumerHook, self).__init__(None) + self.conn = self.get_connection(conn_id) + self.server = None + self.consumer = None + self.producer = None + self.topic = topic + + def get_conn(self): + conf = self.conn.extra_dejson + host = self.conn.host or self.default_host + port = self.conn.port or self.default_port + + conf['enable_auto_commit'] = False + self.server = '{host}:{port}'.format(**locals()) + self.producer = KafkaProducer( + bootstrap_servers=self.server, **conf) + + return self.producer + + def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): + producer = self.get_pub() + future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, + timestamp_ms=timestamp_ms) + producer.close() + return future_record_metadata + + def __repr__(self): + """ + Pretty the hook with the connection info + """ + connected = self.produer is not None + return '' % \ + (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py new file mode 100644 index 0000000000000..738e63c76c730 --- /dev/null +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -0,0 +1,50 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import logging + +from src.kafka_hook import KafkaConsumerHook +from airflow.operators.sensors import BaseSensorOperator +from airflow.utils import apply_defaults + + +class KafkaSensor(BaseSensorOperator): + """ + Consumes the Kafka message with the specific topic + """ + + @apply_defaults + def __init__(self, conn_id, topic, *args, **kwargs): + """ + Initialize the sensor, the connection establish + is put off to it's first time usage. + :param conn_id: + the kafka broker connection whom this sensor + subscripts against. + :param topic: + the subscribed topic + """ + self.topic = topic + self.hook = KafkaConsumerHook(conn_id, topic) + super(KafkaSensor, self).__init__(*args, **kwargs) + + def poke(self, context): + logging.info( + 'Poking topic: %s, using hook: %s', + self.topic, self.hook) + + messages = self.hook.get_messages() + + logging.info( + 'Got messages during poking: %s', messages) + + return messages or False From cbfd33d35067b1beed5a3f90ffdfd2af054a3828 Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 18 Feb 2020 09:05:42 -0500 Subject: [PATCH 148/162] Modifying files in accordance with peer review --- airflow/contrib/hooks/kafka_consumer_hook.py | 24 +++++++++++--------- airflow/contrib/hooks/kafka_producer_hook.py | 24 +++++++++++--------- airflow/contrib/sensors/kafka_sensor.py | 16 +++++++++---- 3 files changed, 38 insertions(+), 26 deletions(-) diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py index 335c267b7837f..1143e2ccfb6a7 100644 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ b/airflow/contrib/hooks/kafka_consumer_hook.py @@ -16,8 +16,8 @@ class KafkaConsumerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): super(KafkaConsumerHook, self).__init__(None) @@ -29,21 +29,21 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT # Disable auto commit as the hook will commit right # after polling. conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.consumer = KafkaConsumer( self.topic, bootstrap_servers=self.server, **conf) return self.consumer - def get_messages(self): + def get_messages(self, timeout_ms=50): """ Get all the messages haven't been consumed, it doesn't block by default, then commit the offset. @@ -51,12 +51,14 @@ def get_messages(self): A list of messages """ consumer = self.get_conn() + try: + # `poll` returns a dict where keys are the partitions + # and values are the corresponding messages. + messages = consumer.poll(timeout_ms) - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms=50) - - consumer.commit() + consumer.commit() + finally: + consumer.close() return messages def __repr__(self): diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py index ffa5d4693e9c2..9c1ebb158afe5 100644 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ b/airflow/contrib/hooks/kafka_producer_hook.py @@ -16,11 +16,11 @@ class KafkaProducerHook(BaseHook): - default_host = 'localhost' - default_port = 9092 + DEFAULT_HOST = 'localhost' + DEFAULT_PORT = 9092 def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) + super(KafkaProducerHook, self).__init__(None) self.conn = self.get_connection(conn_id) self.server = None self.consumer = None @@ -29,21 +29,23 @@ def __init__(self, conn_id, topic): def get_conn(self): conf = self.conn.extra_dejson - host = self.conn.host or self.default_host - port = self.conn.port or self.default_port + host = self.conn.host or self.DEFAULT_HOST + port = self.conn.port or self.DEFAULT_PORT conf['enable_auto_commit'] = False - self.server = '{host}:{port}'.format(**locals()) + self.server = f"""{host}:{port}""" self.producer = KafkaProducer( bootstrap_servers=self.server, **conf) return self.producer - def send_message(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): - producer = self.get_pub() - future_record_metadata = producer.send(topic, value=value, key=key, headers=headers, partition=partition, - timestamp_ms=timestamp_ms) - producer.close() + def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): + producer = self.get_conn() + try: + future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, + timestamp_ms=timestamp_ms) + finally: + producer.close() return future_record_metadata def __repr__(self): diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py index 738e63c76c730..e51328cdd63a9 100644 --- a/airflow/contrib/sensors/kafka_sensor.py +++ b/airflow/contrib/sensors/kafka_sensor.py @@ -12,9 +12,11 @@ import logging -from src.kafka_hook import KafkaConsumerHook -from airflow.operators.sensors import BaseSensorOperator -from airflow.utils import apply_defaults +from werkzeug.utils import cached_property + +from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class KafkaSensor(BaseSensorOperator): @@ -22,6 +24,8 @@ class KafkaSensor(BaseSensorOperator): Consumes the Kafka message with the specific topic """ + templated_fields = ('topic',) + @apply_defaults def __init__(self, conn_id, topic, *args, **kwargs): """ @@ -34,9 +38,13 @@ def __init__(self, conn_id, topic, *args, **kwargs): the subscribed topic """ self.topic = topic - self.hook = KafkaConsumerHook(conn_id, topic) + self.conn_id = conn_id super(KafkaSensor, self).__init__(*args, **kwargs) + @cached_property + def hook(self): + return KafkaConsumerHook(self.conn_id, self.topic) + def poke(self, context): logging.info( 'Poking topic: %s, using hook: %s', From 62f3c57ed6907886817a755906724b428a0fd5c3 Mon Sep 17 00:00:00 2001 From: dan Date: Sat, 23 May 2020 14:46:59 -0400 Subject: [PATCH 149/162] Removed unused imports --- airflow/providers/apache/kafka/hooks/kafka_producer_hook.py | 1 - 1 file changed, 1 deletion(-) diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py index 9919505b4a24d..f6d9fd2428ec0 100644 --- a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py +++ b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py @@ -17,7 +17,6 @@ # under the License. from kafka import KafkaProducer -from kafka.producer.future import FutureRecordMetadata from airflow.hooks.base_hook import BaseHook From b69e5eee05d1711e07ba27868aab199a75bf6a97 Mon Sep 17 00:00:00 2001 From: dan Date: Thu, 30 Apr 2020 09:57:14 -0400 Subject: [PATCH 150/162] Fixed isort issue --- airflow/providers/docker/example_dags/example_docker_swarm.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/airflow/providers/docker/example_dags/example_docker_swarm.py b/airflow/providers/docker/example_dags/example_docker_swarm.py index a4f50917be5bd..3930482ec1fa1 100644 --- a/airflow/providers/docker/example_dags/example_docker_swarm.py +++ b/airflow/providers/docker/example_dags/example_docker_swarm.py @@ -19,8 +19,10 @@ from datetime import timedelta from airflow import DAG +from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor from airflow.providers.docker.operators.docker_swarm import DockerSwarmOperator from airflow.utils.dates import days_ago +from airflow.utils.timezone import datetime default_args = { 'owner': 'airflow', From 5924491c6b47aad9b0f5bfabca5c7ee82c8c737e Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 14:18:52 -0400 Subject: [PATCH 151/162] adding more weird local files --- scripts/ci/docker-compose/local.yml | 1 - 1 file changed, 1 deletion(-) mode change 100644 => 100755 scripts/ci/docker-compose/local.yml diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml old mode 100644 new mode 100755 index 66339a6b40462..6cd05e9ffe739 --- a/scripts/ci/docker-compose/local.yml +++ b/scripts/ci/docker-compose/local.yml @@ -14,7 +14,6 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. ---- version: "2.2" services: airflow: From 7708186532fe9230a518b8e825ae726acc2b63aa Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 14:21:35 -0400 Subject: [PATCH 152/162] Modifying readme to trigger builds --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 647b9f9ac4599..c2869acb4661f 100644 --- a/README.md +++ b/README.md @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. --> - # Apache Airflow [![PyPI version](https://badge.fury.io/py/apache-airflow.svg)](https://badge.fury.io/py/apache-airflow) From 6edd9218649135e475b833934a82ff45c3f61309 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 14:52:39 -0400 Subject: [PATCH 153/162] Updated readme to trigger build --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index c2869acb4661f..647b9f9ac4599 100644 --- a/README.md +++ b/README.md @@ -16,6 +16,7 @@ specific language governing permissions and limitations under the License. --> + # Apache Airflow [![PyPI version](https://badge.fury.io/py/apache-airflow.svg)](https://badge.fury.io/py/apache-airflow) From bf1217233c78ace02e743545fcebf00c7da85909 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 21:04:17 -0400 Subject: [PATCH 154/162] Fixed yaml linting issues --- scripts/ci/docker-compose/local.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml index 6cd05e9ffe739..6a521f9533de7 100755 --- a/scripts/ci/docker-compose/local.yml +++ b/scripts/ci/docker-compose/local.yml @@ -56,6 +56,9 @@ services: - ../../../kubernetes_tests:/opt/airflow/kubernetes_tests:cached - ../../../tmp:/tmp:cached # END automatically generated volumes from LOCAL_MOUNTS in _local_mounts.sh + - ../../../tmp:/opt/airflow/tmp:cached + # END automatically generated volumes from LOCAL_MOUNTS in + # _local_mounts.sh environment: - HOST_USER_ID - HOST_GROUP_ID From f2b7f977640df46d3c96c4e4bb385ba28754b94c Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 14:21:35 -0400 Subject: [PATCH 155/162] Modifying readme to trigger builds --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 647b9f9ac4599..c2869acb4661f 100644 --- a/README.md +++ b/README.md @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. --> - # Apache Airflow [![PyPI version](https://badge.fury.io/py/apache-airflow.svg)](https://badge.fury.io/py/apache-airflow) From 3a24cc16c7212137b6d2f91882d8bdbe5e4f624f Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 14:52:39 -0400 Subject: [PATCH 156/162] Updated readme to trigger build --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index c2869acb4661f..647b9f9ac4599 100644 --- a/README.md +++ b/README.md @@ -16,6 +16,7 @@ specific language governing permissions and limitations under the License. --> + # Apache Airflow [![PyPI version](https://badge.fury.io/py/apache-airflow.svg)](https://badge.fury.io/py/apache-airflow) From a03c99295f0304ed610dfe21196bc60e1b29e529 Mon Sep 17 00:00:00 2001 From: dan Date: Fri, 5 Jun 2020 17:38:27 -0400 Subject: [PATCH 157/162] updating dependencies files --- airflow/providers/docker/example_dags/example_docker_swarm.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/airflow/providers/docker/example_dags/example_docker_swarm.py b/airflow/providers/docker/example_dags/example_docker_swarm.py index 3930482ec1fa1..a4f50917be5bd 100644 --- a/airflow/providers/docker/example_dags/example_docker_swarm.py +++ b/airflow/providers/docker/example_dags/example_docker_swarm.py @@ -19,10 +19,8 @@ from datetime import timedelta from airflow import DAG -from airflow.providers.apache.kafka.sensors.kafka_sensor import KafkaSensor from airflow.providers.docker.operators.docker_swarm import DockerSwarmOperator from airflow.utils.dates import days_ago -from airflow.utils.timezone import datetime default_args = { 'owner': 'airflow', From 3e3205067f1163ce2f089d11c3735442d8628e04 Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 17 Jun 2020 09:53:14 -0400 Subject: [PATCH 158/162] Modified file perms --- scripts/ci/docker-compose/local.yml | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 scripts/ci/docker-compose/local.yml diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml old mode 100755 new mode 100644 From f9f1fb8caa188d8bc8234afe63c9db557d9a61ac Mon Sep 17 00:00:00 2001 From: dan Date: Tue, 25 Aug 2020 19:59:25 -0400 Subject: [PATCH 159/162] Removed legacy hook and sensor code --- airflow/contrib/hooks/kafka_consumer_hook.py | 71 -------------------- airflow/contrib/hooks/kafka_producer_hook.py | 58 ---------------- airflow/contrib/sensors/kafka_sensor.py | 58 ---------------- 3 files changed, 187 deletions(-) delete mode 100644 airflow/contrib/hooks/kafka_consumer_hook.py delete mode 100644 airflow/contrib/hooks/kafka_producer_hook.py delete mode 100644 airflow/contrib/sensors/kafka_sensor.py diff --git a/airflow/contrib/hooks/kafka_consumer_hook.py b/airflow/contrib/hooks/kafka_consumer_hook.py deleted file mode 100644 index 1143e2ccfb6a7..0000000000000 --- a/airflow/contrib/hooks/kafka_consumer_hook.py +++ /dev/null @@ -1,71 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaConsumer - - -class KafkaConsumerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaConsumerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - # Disable auto commit as the hook will commit right - # after polling. - conf['enable_auto_commit'] = False - - self.server = f"""{host}:{port}""" - self.consumer = KafkaConsumer( - self.topic, - bootstrap_servers=self.server, **conf) - - return self.consumer - - def get_messages(self, timeout_ms=50): - """ - Get all the messages haven't been consumed, it doesn't - block by default, then commit the offset. - :return: - A list of messages - """ - consumer = self.get_conn() - try: - # `poll` returns a dict where keys are the partitions - # and values are the corresponding messages. - messages = consumer.poll(timeout_ms) - - consumer.commit() - finally: - consumer.close() - return messages - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.consumer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/hooks/kafka_producer_hook.py b/airflow/contrib/hooks/kafka_producer_hook.py deleted file mode 100644 index 9c1ebb158afe5..0000000000000 --- a/airflow/contrib/hooks/kafka_producer_hook.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -from airflow.hooks import base_hook as BaseHook -from kafka import KafkaProducer - - -class KafkaProducerHook(BaseHook): - - DEFAULT_HOST = 'localhost' - DEFAULT_PORT = 9092 - - def __init__(self, conn_id, topic): - super(KafkaProducerHook, self).__init__(None) - self.conn = self.get_connection(conn_id) - self.server = None - self.consumer = None - self.producer = None - self.topic = topic - - def get_conn(self): - conf = self.conn.extra_dejson - host = self.conn.host or self.DEFAULT_HOST - port = self.conn.port or self.DEFAULT_PORT - - conf['enable_auto_commit'] = False - self.server = f"""{host}:{port}""" - self.producer = KafkaProducer( - bootstrap_servers=self.server, **conf) - - return self.producer - - def send_message(self, topic, value=None, key=None, partition=None, timestamp_ms=None): - producer = self.get_conn() - try: - future_record_metadata = producer.send(topic, value=value, key=key, partition=partition, - timestamp_ms=timestamp_ms) - finally: - producer.close() - return future_record_metadata - - def __repr__(self): - """ - Pretty the hook with the connection info - """ - connected = self.produer is not None - return '' % \ - (connected, self.server, self.topic) diff --git a/airflow/contrib/sensors/kafka_sensor.py b/airflow/contrib/sensors/kafka_sensor.py deleted file mode 100644 index e51328cdd63a9..0000000000000 --- a/airflow/contrib/sensors/kafka_sensor.py +++ /dev/null @@ -1,58 +0,0 @@ -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -import logging - -from werkzeug.utils import cached_property - -from airflow.contrib.hooks.kafka_consumer_hook import KafkaConsumerHook -from airflow.sensors.base_sensor_operator import BaseSensorOperator -from airflow.utils.decorators import apply_defaults - - -class KafkaSensor(BaseSensorOperator): - """ - Consumes the Kafka message with the specific topic - """ - - templated_fields = ('topic',) - - @apply_defaults - def __init__(self, conn_id, topic, *args, **kwargs): - """ - Initialize the sensor, the connection establish - is put off to it's first time usage. - :param conn_id: - the kafka broker connection whom this sensor - subscripts against. - :param topic: - the subscribed topic - """ - self.topic = topic - self.conn_id = conn_id - super(KafkaSensor, self).__init__(*args, **kwargs) - - @cached_property - def hook(self): - return KafkaConsumerHook(self.conn_id, self.topic) - - def poke(self, context): - logging.info( - 'Poking topic: %s, using hook: %s', - self.topic, self.hook) - - messages = self.hook.get_messages() - - logging.info( - 'Got messages during poking: %s', messages) - - return messages or False From f26f88056c2af3601a41097248cec7565db140ec Mon Sep 17 00:00:00 2001 From: dan Date: Wed, 26 Aug 2020 15:36:14 -0400 Subject: [PATCH 160/162] ?????? --- BREEZE.rst | 0 CHANGELOG.txt | 0 CI.rst | 0 CODE_OF_CONDUCT.md | 0 CONTRIBUTING.rst | 0 Dockerfile | 0 Dockerfile.ci | 0 IMAGES.rst | 0 INSTALL | 0 LICENSE | 0 LOCAL_VIRTUALENV.rst | 0 MANIFEST.in | 0 NOTICE | 0 README.md | 0 STATIC_CODE_CHECKS.rst | 0 TESTING.rst | 0 UPDATING.md | 0 airflow/__init__.py | 0 airflow/__main__.py | 0 airflow/alembic.ini | 0 airflow/api/__init__.py | 0 airflow/api/auth/__init__.py | 0 airflow/api/auth/backend/__init__.py | 0 airflow/api/auth/backend/basic_auth.py | 0 airflow/api/auth/backend/default.py | 0 airflow/api/auth/backend/deny_all.py | 0 airflow/api/auth/backend/kerberos_auth.py | 0 airflow/api/client/__init__.py | 0 airflow/api/client/api_client.py | 0 airflow/api/client/json_client.py | 0 airflow/api/client/local_client.py | 0 airflow/api/common/__init__.py | 0 airflow/api/common/experimental/__init__.py | 0 airflow/api/common/experimental/delete_dag.py | 0 airflow/api/common/experimental/get_code.py | 0 .../common/experimental/get_dag_run_state.py | 0 airflow/api/common/experimental/get_dag_runs.py | 0 airflow/api/common/experimental/get_lineage.py | 0 airflow/api/common/experimental/get_task.py | 0 .../common/experimental/get_task_instance.py | 0 airflow/api/common/experimental/mark_tasks.py | 0 airflow/api/common/experimental/pool.py | 0 airflow/api/common/experimental/trigger_dag.py | 0 airflow/api_connexion/__init__.py | 0 airflow/api_connexion/endpoints/__init__.py | 0 .../api_connexion/endpoints/config_endpoint.py | 0 .../endpoints/connection_endpoint.py | 0 airflow/api_connexion/endpoints/dag_endpoint.py | 0 .../api_connexion/endpoints/dag_run_endpoint.py | 0 .../endpoints/dag_source_endpoint.py | 0 .../endpoints/event_log_endpoint.py | 0 .../endpoints/extra_link_endpoint.py | 0 .../api_connexion/endpoints/health_endpoint.py | 0 .../endpoints/import_error_endpoint.py | 0 airflow/api_connexion/endpoints/log_endpoint.py | 0 .../api_connexion/endpoints/pool_endpoint.py | 0 .../api_connexion/endpoints/task_endpoint.py | 0 .../endpoints/task_instance_endpoint.py | 0 .../endpoints/variable_endpoint.py | 0 .../api_connexion/endpoints/version_endpoint.py | 0 .../api_connexion/endpoints/xcom_endpoint.py | 0 airflow/api_connexion/exceptions.py | 0 airflow/api_connexion/openapi/v1.yaml | 0 airflow/api_connexion/parameters.py | 0 airflow/api_connexion/schemas/__init__.py | 0 airflow/api_connexion/schemas/common_schema.py | 0 airflow/api_connexion/schemas/config_schema.py | 0 .../api_connexion/schemas/connection_schema.py | 0 airflow/api_connexion/schemas/dag_run_schema.py | 0 airflow/api_connexion/schemas/dag_schema.py | 0 .../api_connexion/schemas/dag_source_schema.py | 0 airflow/api_connexion/schemas/enum_schemas.py | 0 airflow/api_connexion/schemas/error_schema.py | 0 .../api_connexion/schemas/event_log_schema.py | 0 airflow/api_connexion/schemas/health_schema.py | 0 airflow/api_connexion/schemas/log_schema.py | 0 airflow/api_connexion/schemas/pool_schema.py | 0 airflow/api_connexion/schemas/task_schema.py | 0 .../api_connexion/schemas/variable_schema.py | 0 airflow/api_connexion/schemas/version_schema.py | 0 airflow/api_connexion/schemas/xcom_schema.py | 0 airflow/api_connexion/security.py | 0 airflow/cli/__init__.py | 0 airflow/cli/cli_parser.py | 0 airflow/cli/commands/__init__.py | 0 airflow/cli/commands/celery_command.py | 0 airflow/cli/commands/config_command.py | 0 airflow/cli/commands/connection_command.py | 0 airflow/cli/commands/dag_command.py | 0 airflow/cli/commands/db_command.py | 0 airflow/cli/commands/info_command.py | 0 airflow/cli/commands/kerberos_command.py | 0 airflow/cli/commands/legacy_commands.py | 0 airflow/cli/commands/plugins_command.py | 0 airflow/cli/commands/pool_command.py | 0 airflow/cli/commands/role_command.py | 0 .../cli/commands/rotate_fernet_key_command.py | 0 airflow/cli/commands/scheduler_command.py | 0 airflow/cli/commands/sync_perm_command.py | 0 airflow/cli/commands/task_command.py | 0 airflow/cli/commands/user_command.py | 0 airflow/cli/commands/variable_command.py | 0 airflow/cli/commands/version_command.py | 0 airflow/cli/commands/webserver_command.py | 0 airflow/config_templates/__init__.py | 0 .../config_templates/airflow_local_settings.py | 0 airflow/config_templates/config.yml | 0 airflow/config_templates/default_airflow.cfg | 0 airflow/config_templates/default_celery.py | 0 airflow/config_templates/default_test.cfg | 0 .../default_webserver_config.py | 0 airflow/configuration.py | 0 airflow/contrib/__init__.py | 0 airflow/contrib/hooks/__init__.py | 0 airflow/contrib/hooks/aws_athena_hook.py | 0 airflow/contrib/hooks/aws_datasync_hook.py | 0 airflow/contrib/hooks/aws_dynamodb_hook.py | 0 airflow/contrib/hooks/aws_firehose_hook.py | 0 airflow/contrib/hooks/aws_glue_catalog_hook.py | 0 airflow/contrib/hooks/aws_hook.py | 0 airflow/contrib/hooks/aws_lambda_hook.py | 0 airflow/contrib/hooks/aws_logs_hook.py | 0 airflow/contrib/hooks/aws_sns_hook.py | 0 airflow/contrib/hooks/aws_sqs_hook.py | 0 .../hooks/azure_container_instance_hook.py | 0 .../hooks/azure_container_registry_hook.py | 0 .../hooks/azure_container_volume_hook.py | 0 airflow/contrib/hooks/azure_cosmos_hook.py | 0 airflow/contrib/hooks/azure_data_lake_hook.py | 0 airflow/contrib/hooks/azure_fileshare_hook.py | 0 airflow/contrib/hooks/bigquery_hook.py | 0 airflow/contrib/hooks/cassandra_hook.py | 0 airflow/contrib/hooks/cloudant_hook.py | 0 airflow/contrib/hooks/databricks_hook.py | 0 airflow/contrib/hooks/datadog_hook.py | 0 airflow/contrib/hooks/datastore_hook.py | 0 airflow/contrib/hooks/dingding_hook.py | 0 airflow/contrib/hooks/discord_webhook_hook.py | 0 airflow/contrib/hooks/emr_hook.py | 0 airflow/contrib/hooks/fs_hook.py | 0 airflow/contrib/hooks/ftp_hook.py | 0 airflow/contrib/hooks/gcp_api_base_hook.py | 0 airflow/contrib/hooks/gcp_bigtable_hook.py | 0 airflow/contrib/hooks/gcp_cloud_build_hook.py | 0 airflow/contrib/hooks/gcp_compute_hook.py | 0 airflow/contrib/hooks/gcp_container_hook.py | 0 airflow/contrib/hooks/gcp_dataflow_hook.py | 0 airflow/contrib/hooks/gcp_dataproc_hook.py | 0 airflow/contrib/hooks/gcp_dlp_hook.py | 0 airflow/contrib/hooks/gcp_function_hook.py | 0 airflow/contrib/hooks/gcp_kms_hook.py | 0 airflow/contrib/hooks/gcp_mlengine_hook.py | 0 .../contrib/hooks/gcp_natural_language_hook.py | 0 airflow/contrib/hooks/gcp_pubsub_hook.py | 0 airflow/contrib/hooks/gcp_spanner_hook.py | 0 .../contrib/hooks/gcp_speech_to_text_hook.py | 0 airflow/contrib/hooks/gcp_sql_hook.py | 0 airflow/contrib/hooks/gcp_tasks_hook.py | 0 .../contrib/hooks/gcp_text_to_speech_hook.py | 0 airflow/contrib/hooks/gcp_transfer_hook.py | 0 airflow/contrib/hooks/gcp_translate_hook.py | 0 .../hooks/gcp_video_intelligence_hook.py | 0 airflow/contrib/hooks/gcp_vision_hook.py | 0 airflow/contrib/hooks/gcs_hook.py | 0 airflow/contrib/hooks/gdrive_hook.py | 0 airflow/contrib/hooks/grpc_hook.py | 0 airflow/contrib/hooks/imap_hook.py | 0 airflow/contrib/hooks/jenkins_hook.py | 0 airflow/contrib/hooks/jira_hook.py | 0 airflow/contrib/hooks/mongo_hook.py | 0 airflow/contrib/hooks/openfaas_hook.py | 0 airflow/contrib/hooks/opsgenie_alert_hook.py | 0 airflow/contrib/hooks/pagerduty_hook.py | 0 airflow/contrib/hooks/pinot_hook.py | 0 airflow/contrib/hooks/qubole_check_hook.py | 0 airflow/contrib/hooks/qubole_hook.py | 0 airflow/contrib/hooks/redis_hook.py | 0 airflow/contrib/hooks/redshift_hook.py | 0 airflow/contrib/hooks/sagemaker_hook.py | 0 airflow/contrib/hooks/salesforce_hook.py | 0 airflow/contrib/hooks/segment_hook.py | 0 airflow/contrib/hooks/sftp_hook.py | 0 airflow/contrib/hooks/slack_webhook_hook.py | 0 airflow/contrib/hooks/snowflake_hook.py | 0 airflow/contrib/hooks/spark_jdbc_hook.py | 0 airflow/contrib/hooks/spark_sql_hook.py | 0 airflow/contrib/hooks/spark_submit_hook.py | 0 airflow/contrib/hooks/sqoop_hook.py | 0 airflow/contrib/hooks/ssh_hook.py | 0 airflow/contrib/hooks/vertica_hook.py | 0 airflow/contrib/hooks/wasb_hook.py | 0 airflow/contrib/hooks/winrm_hook.py | 0 airflow/contrib/operators/__init__.py | 0 airflow/contrib/operators/adls_list_operator.py | 0 airflow/contrib/operators/adls_to_gcs.py | 0 .../contrib/operators/aws_athena_operator.py | 0 .../operators/aws_sqs_publish_operator.py | 0 airflow/contrib/operators/awsbatch_operator.py | 0 .../azure_container_instances_operator.py | 0 .../contrib/operators/azure_cosmos_operator.py | 0 .../operators/bigquery_check_operator.py | 0 airflow/contrib/operators/bigquery_get_data.py | 0 airflow/contrib/operators/bigquery_operator.py | 0 .../operators/bigquery_table_delete_operator.py | 0 .../contrib/operators/bigquery_to_bigquery.py | 0 airflow/contrib/operators/bigquery_to_gcs.py | 0 .../operators/bigquery_to_mysql_operator.py | 0 airflow/contrib/operators/cassandra_to_gcs.py | 0 .../contrib/operators/databricks_operator.py | 0 airflow/contrib/operators/dataflow_operator.py | 0 airflow/contrib/operators/dataproc_operator.py | 0 .../operators/datastore_export_operator.py | 0 .../operators/datastore_import_operator.py | 0 airflow/contrib/operators/dingding_operator.py | 0 .../operators/discord_webhook_operator.py | 0 .../contrib/operators/docker_swarm_operator.py | 0 airflow/contrib/operators/druid_operator.py | 0 airflow/contrib/operators/dynamodb_to_s3.py | 0 airflow/contrib/operators/ecs_operator.py | 0 .../contrib/operators/emr_add_steps_operator.py | 0 .../operators/emr_create_job_flow_operator.py | 0 .../emr_terminate_job_flow_operator.py | 0 airflow/contrib/operators/file_to_gcs.py | 0 airflow/contrib/operators/file_to_wasb.py | 0 .../contrib/operators/gcp_bigtable_operator.py | 0 .../operators/gcp_cloud_build_operator.py | 0 .../contrib/operators/gcp_compute_operator.py | 0 .../contrib/operators/gcp_container_operator.py | 0 airflow/contrib/operators/gcp_dlp_operator.py | 0 .../contrib/operators/gcp_function_operator.py | 0 .../operators/gcp_natural_language_operator.py | 0 .../contrib/operators/gcp_spanner_operator.py | 0 .../operators/gcp_speech_to_text_operator.py | 0 airflow/contrib/operators/gcp_sql_operator.py | 0 airflow/contrib/operators/gcp_tasks_operator.py | 0 .../operators/gcp_text_to_speech_operator.py | 0 .../contrib/operators/gcp_transfer_operator.py | 0 .../contrib/operators/gcp_translate_operator.py | 0 .../operators/gcp_translate_speech_operator.py | 0 .../gcp_video_intelligence_operator.py | 0 .../contrib/operators/gcp_vision_operator.py | 0 airflow/contrib/operators/gcs_acl_operator.py | 0 .../contrib/operators/gcs_delete_operator.py | 0 .../contrib/operators/gcs_download_operator.py | 0 airflow/contrib/operators/gcs_list_operator.py | 0 airflow/contrib/operators/gcs_operator.py | 0 airflow/contrib/operators/gcs_to_bq.py | 0 airflow/contrib/operators/gcs_to_gcs.py | 0 .../operators/gcs_to_gcs_transfer_operator.py | 0 .../contrib/operators/gcs_to_gdrive_operator.py | 0 airflow/contrib/operators/gcs_to_s3.py | 0 airflow/contrib/operators/grpc_operator.py | 0 airflow/contrib/operators/hive_to_dynamodb.py | 0 .../operators/imap_attachment_to_s3_operator.py | 0 .../operators/jenkins_job_trigger_operator.py | 0 airflow/contrib/operators/jira_operator.py | 0 .../operators/kubernetes_pod_operator.py | 0 airflow/contrib/operators/mlengine_operator.py | 0 airflow/contrib/operators/mongo_to_s3.py | 0 airflow/contrib/operators/mssql_to_gcs.py | 0 airflow/contrib/operators/mysql_to_gcs.py | 0 .../operators/opsgenie_alert_operator.py | 0 .../oracle_to_azure_data_lake_transfer.py | 0 .../operators/oracle_to_oracle_transfer.py | 0 .../operators/postgres_to_gcs_operator.py | 0 airflow/contrib/operators/pubsub_operator.py | 0 .../contrib/operators/qubole_check_operator.py | 0 airflow/contrib/operators/qubole_operator.py | 0 .../contrib/operators/redis_publish_operator.py | 0 .../operators/s3_copy_object_operator.py | 0 .../operators/s3_delete_objects_operator.py | 0 airflow/contrib/operators/s3_list_operator.py | 0 airflow/contrib/operators/s3_to_gcs_operator.py | 0 .../operators/s3_to_gcs_transfer_operator.py | 0 .../contrib/operators/s3_to_sftp_operator.py | 0 .../operators/sagemaker_base_operator.py | 0 .../sagemaker_endpoint_config_operator.py | 0 .../operators/sagemaker_endpoint_operator.py | 0 .../operators/sagemaker_model_operator.py | 0 .../operators/sagemaker_training_operator.py | 0 .../operators/sagemaker_transform_operator.py | 0 .../operators/sagemaker_tuning_operator.py | 0 .../operators/segment_track_event_operator.py | 0 airflow/contrib/operators/sftp_operator.py | 0 .../contrib/operators/sftp_to_s3_operator.py | 0 .../contrib/operators/slack_webhook_operator.py | 0 airflow/contrib/operators/snowflake_operator.py | 0 .../contrib/operators/sns_publish_operator.py | 0 .../contrib/operators/spark_jdbc_operator.py | 0 airflow/contrib/operators/spark_sql_operator.py | 0 .../contrib/operators/spark_submit_operator.py | 0 airflow/contrib/operators/sql_to_gcs.py | 0 airflow/contrib/operators/sqoop_operator.py | 0 airflow/contrib/operators/ssh_operator.py | 0 airflow/contrib/operators/vertica_operator.py | 0 airflow/contrib/operators/vertica_to_hive.py | 0 airflow/contrib/operators/vertica_to_mysql.py | 0 .../operators/wasb_delete_blob_operator.py | 0 airflow/contrib/operators/winrm_operator.py | 0 airflow/contrib/secrets/__init__.py | 0 airflow/contrib/secrets/aws_secrets_manager.py | 0 airflow/contrib/secrets/aws_systems_manager.py | 0 airflow/contrib/secrets/gcp_secrets_manager.py | 0 airflow/contrib/secrets/hashicorp_vault.py | 0 airflow/contrib/sensors/__init__.py | 0 airflow/contrib/sensors/aws_athena_sensor.py | 0 .../aws_glue_catalog_partition_sensor.py | 0 .../sensors/aws_redshift_cluster_sensor.py | 0 airflow/contrib/sensors/aws_sqs_sensor.py | 0 airflow/contrib/sensors/azure_cosmos_sensor.py | 0 airflow/contrib/sensors/bash_sensor.py | 0 airflow/contrib/sensors/bigquery_sensor.py | 0 .../contrib/sensors/cassandra_record_sensor.py | 0 .../contrib/sensors/cassandra_table_sensor.py | 0 airflow/contrib/sensors/celery_queue_sensor.py | 0 airflow/contrib/sensors/datadog_sensor.py | 0 airflow/contrib/sensors/emr_base_sensor.py | 0 airflow/contrib/sensors/emr_job_flow_sensor.py | 0 airflow/contrib/sensors/emr_step_sensor.py | 0 airflow/contrib/sensors/file_sensor.py | 0 airflow/contrib/sensors/ftp_sensor.py | 0 airflow/contrib/sensors/gcp_transfer_sensor.py | 0 airflow/contrib/sensors/gcs_sensor.py | 0 airflow/contrib/sensors/hdfs_sensor.py | 0 .../contrib/sensors/imap_attachment_sensor.py | 0 airflow/contrib/sensors/jira_sensor.py | 0 airflow/contrib/sensors/mongo_sensor.py | 0 airflow/contrib/sensors/pubsub_sensor.py | 0 airflow/contrib/sensors/python_sensor.py | 0 airflow/contrib/sensors/qubole_sensor.py | 0 airflow/contrib/sensors/redis_key_sensor.py | 0 airflow/contrib/sensors/redis_pub_sub_sensor.py | 0 .../contrib/sensors/sagemaker_base_sensor.py | 0 .../sensors/sagemaker_endpoint_sensor.py | 0 .../sensors/sagemaker_training_sensor.py | 0 .../sensors/sagemaker_transform_sensor.py | 0 .../contrib/sensors/sagemaker_tuning_sensor.py | 0 airflow/contrib/sensors/sftp_sensor.py | 0 airflow/contrib/sensors/wasb_sensor.py | 0 airflow/contrib/sensors/weekday_sensor.py | 0 airflow/contrib/task_runner/__init__.py | 0 .../contrib/task_runner/cgroup_task_runner.py | 0 airflow/contrib/utils/__init__.py | 0 airflow/contrib/utils/gcp_field_sanitizer.py | 0 airflow/contrib/utils/gcp_field_validator.py | 0 airflow/contrib/utils/log/__init__.py | 0 .../log/task_handler_with_custom_formatter.py | 0 .../contrib/utils/mlengine_operator_utils.py | 0 .../utils/mlengine_prediction_summary.py | 0 airflow/contrib/utils/sendgrid.py | 0 airflow/contrib/utils/weekday.py | 0 airflow/dag/__init__.py | 0 airflow/dag/base_dag.py | 0 airflow/decorators.py | 0 airflow/example_dags/__init__.py | 0 airflow/example_dags/example_bash_operator.py | 0 airflow/example_dags/example_branch_operator.py | 0 .../example_branch_python_dop_operator_3.py | 0 airflow/example_dags/example_complex.py | 0 .../example_external_task_marker_dag.py | 0 .../example_dags/example_kubernetes_executor.py | 0 .../example_kubernetes_executor_config.py | 0 airflow/example_dags/example_latest_only.py | 0 .../example_latest_only_with_trigger.py | 0 .../example_dags/example_nested_branch_dag.py | 0 .../example_passing_params_via_test_command.py | 0 airflow/example_dags/example_python_operator.py | 0 .../example_short_circuit_operator.py | 0 airflow/example_dags/example_skip_dag.py | 0 airflow/example_dags/example_subdag_operator.py | 0 .../example_trigger_controller_dag.py | 0 .../example_dags/example_trigger_target_dag.py | 0 airflow/example_dags/example_xcom.py | 0 airflow/example_dags/example_xcomargs.py | 0 airflow/example_dags/input_notebook.ipynb | 0 airflow/example_dags/libs/__init__.py | 0 airflow/example_dags/libs/helper.py | 0 airflow/example_dags/subdags/__init__.py | 0 airflow/example_dags/subdags/subdag.py | 0 airflow/example_dags/test_utils.py | 0 airflow/example_dags/tutorial.py | 0 airflow/exceptions.py | 0 airflow/executors/__init__.py | 0 airflow/executors/base_executor.py | 0 airflow/executors/celery_executor.py | 0 airflow/executors/dask_executor.py | 0 airflow/executors/debug_executor.py | 0 airflow/executors/executor_loader.py | 0 airflow/executors/kubernetes_executor.py | 0 airflow/executors/local_executor.py | 0 airflow/executors/sequential_executor.py | 0 airflow/hooks/S3_hook.py | 0 airflow/hooks/__init__.py | 0 airflow/hooks/base_hook.py | 0 airflow/hooks/dbapi_hook.py | 0 airflow/hooks/docker_hook.py | 0 airflow/hooks/druid_hook.py | 0 airflow/hooks/filesystem.py | 0 airflow/hooks/hdfs_hook.py | 0 airflow/hooks/hive_hooks.py | 0 airflow/hooks/http_hook.py | 0 airflow/hooks/jdbc_hook.py | 0 airflow/hooks/mssql_hook.py | 0 airflow/hooks/mysql_hook.py | 0 airflow/hooks/oracle_hook.py | 0 airflow/hooks/pig_hook.py | 0 airflow/hooks/postgres_hook.py | 0 airflow/hooks/presto_hook.py | 0 airflow/hooks/samba_hook.py | 0 airflow/hooks/slack_hook.py | 0 airflow/hooks/sqlite_hook.py | 0 airflow/hooks/webhdfs_hook.py | 0 airflow/hooks/zendesk_hook.py | 0 airflow/jobs/__init__.py | 0 airflow/jobs/backfill_job.py | 0 airflow/jobs/base_job.py | 0 airflow/jobs/local_task_job.py | 0 airflow/jobs/scheduler_job.py | 0 airflow/kubernetes/__init__.py | 0 airflow/kubernetes/k8s_model.py | 0 airflow/kubernetes/kube_client.py | 0 airflow/kubernetes/pod.py | 0 airflow/kubernetes/pod_generator.py | 0 airflow/kubernetes/pod_launcher.py | 0 airflow/kubernetes/pod_runtime_info_env.py | 0 airflow/kubernetes/refresh_config.py | 0 airflow/kubernetes/secret.py | 0 airflow/kubernetes/volume.py | 0 airflow/kubernetes/volume_mount.py | 0 airflow/kubernetes/worker_configuration.py | 0 airflow/lineage/__init__.py | 0 airflow/lineage/entities.py | 0 airflow/logging_config.py | 0 airflow/macros/__init__.py | 0 airflow/macros/hive.py | 0 airflow/migrations/__init__.py | 0 airflow/migrations/env.py | 0 airflow/migrations/script.py.mako | 0 ...4c1210f153_increase_queue_name_size_limit.py | 0 .../versions/03bc53e68815_add_sm_dag_index.py | 0 .../versions/05f30312d566_merge_heads.py | 0 .../0a2a5b66e19d_add_task_reschedule_table.py | 0 .../0e2a74e0fc9f_add_time_zone_awareness.py | 0 ...7_add_dag_id_state_index_on_dag_run_table.py | 0 .../versions/13eb55f81627_for_compatibility.py | 0 .../1507a7289a2f_create_is_encrypted.py | 0 ...09e3_add_is_encrypted_column_to_variable_.py | 0 .../versions/1b38cef5b76e_add_dagrun.py | 0 .../versions/211e584da130_add_ti_state_index.py | 0 ...7c24_add_executor_config_to_task_instance.py | 0 .../versions/2e541a1dcfed_task_duration.py | 0 .../versions/2e82aab8ef20_rename_user_table.py | 0 ...e90f54d61_more_logging_into_task_isntance.py | 0 ...ff4_add_kubernetes_resource_checkpointing.py | 0 .../3c20cacc0044_add_dagrun_run_type.py | 0 .../versions/40e67319e3a9_dagrun_config.py | 0 .../41f5f12752f8_add_superuser_field.py | 0 .../versions/4446e08588_dagrun_start_end.py | 0 ...f1_add_fractional_seconds_to_mysql_tables.py | 0 .../502898887f84_adding_extra_to_log.py | 0 .../versions/52d714495f0_job_id_indices.py | 0 .../561833c1c74b_add_password_column_to_user.py | 0 ...e7d17757c7a_add_pid_field_to_taskinstance.py | 0 ...64de9cddf6c9_add_task_fails_journal_table.py | 0 ...344a4_make_taskinstance_pool_not_nullable.py | 0 ..._change_datetime_to_datetime2_6_on_mssql_.py | 0 .../versions/7939bcff74ba_add_dagtags_table.py | 0 .../8504051e801b_xcom_dag_task_indices.py | 0 ...f_add_rendered_task_instance_fields_table.py | 0 .../856955da8476_fix_sqlite_foreign_key.py | 0 ...215c0_add_kubernetes_scheduler_uniqueness.py | 0 ...763f6d53_add_unique_constraint_to_conn_id.py | 0 ...f966b9c467a_set_conn_type_as_non_nullable.py | 0 ...47c8_task_reschedule_fk_on_cascade_delete.py | 0 .../947454bf1dff_add_ti_job_id_index.py | 0 .../versions/952da73b5eff_add_dag_code_table.py | 0 .../versions/9635ae0956e7_index_faskfail.py | 0 airflow/migrations/versions/__init__.py | 0 ...16b_add_pool_slots_field_to_task_instance.py | 0 .../a56c9515abdc_remove_dag_stat_table.py | 0 ..._add_precision_to_execution_date_in_mysql.py | 0 .../versions/b0125267960b_merge_heads.py | 0 ...b25a55525161_increase_length_of_pool_name.py | 0 .../b3b105409875_add_root_dag_id_to_dag.py | 0 ...896_add_a_column_to_track_the_encryption_.py | 0 ..._add_notification_sent_column_to_sla_miss.py | 0 .../bbf4a7ad0465_remove_id_column_from_xcom.py | 0 ...c56_make_xcom_value_column_a_large_binary.py | 0 .../bf00311e1990_add_index_to_taskinstance.py | 0 .../versions/c8ffec048a3b_add_fields_to_dag.py | 0 ...dc7_add_max_tries_column_to_task_instance.py | 0 .../cf5dc11e79ad_drop_user_and_chart.py | 0 ...d2ae31099d61_increase_text_size_for_mysql.py | 0 .../d38e04c12aa2_add_serialized_dag_table.py | 0 ...5a_add_dag_hash_column_to_serialized_dag_.py | 0 .../versions/dd25f486b8ea_add_idx_log_dag.py | 0 ...dd4ecb8fbee3_add_schedule_interval_to_dag.py | 0 .../versions/e3a246e0dc1_current_schema.py | 0 ...23433877c24_fix_mysql_not_null_constraint.py | 0 .../f2ca10b85618_add_dag_stats_table.py | 0 ...f_increase_length_for_connection_password.py | 0 airflow/models/__init__.py | 0 airflow/models/base.py | 0 airflow/models/baseoperator.py | 0 airflow/models/connection.py | 0 airflow/models/crypto.py | 0 airflow/models/dag.py | 0 airflow/models/dagbag.py | 0 airflow/models/dagcode.py | 0 airflow/models/dagpickle.py | 0 airflow/models/dagrun.py | 0 airflow/models/errors.py | 0 airflow/models/kubernetes.py | 0 airflow/models/log.py | 0 airflow/models/pool.py | 0 airflow/models/renderedtifields.py | 0 airflow/models/serialized_dag.py | 0 airflow/models/skipmixin.py | 0 airflow/models/slamiss.py | 0 airflow/models/taskfail.py | 0 airflow/models/taskinstance.py | 0 airflow/models/taskreschedule.py | 0 airflow/models/variable.py | 0 airflow/models/xcom.py | 0 airflow/models/xcom_arg.py | 0 airflow/mypy/__init__.py | 0 airflow/mypy/plugin/__init__.py | 0 airflow/mypy/plugin/decorators.py | 0 airflow/operators/__init__.py | 0 airflow/operators/bash.py | 0 airflow/operators/bash_operator.py | 0 airflow/operators/branch_operator.py | 0 airflow/operators/check_operator.py | 0 airflow/operators/dagrun_operator.py | 0 airflow/operators/docker_operator.py | 0 airflow/operators/druid_check_operator.py | 0 airflow/operators/dummy_operator.py | 0 airflow/operators/email.py | 0 airflow/operators/email_operator.py | 0 airflow/operators/gcs_to_s3.py | 0 airflow/operators/generic_transfer.py | 0 airflow/operators/google_api_to_s3_transfer.py | 0 airflow/operators/hive_operator.py | 0 airflow/operators/hive_stats_operator.py | 0 airflow/operators/hive_to_druid.py | 0 airflow/operators/hive_to_mysql.py | 0 airflow/operators/hive_to_samba_operator.py | 0 airflow/operators/http_operator.py | 0 airflow/operators/jdbc_operator.py | 0 airflow/operators/latest_only_operator.py | 0 airflow/operators/mssql_operator.py | 0 airflow/operators/mssql_to_hive.py | 0 airflow/operators/mysql_operator.py | 0 airflow/operators/mysql_to_hive.py | 0 airflow/operators/oracle_operator.py | 0 airflow/operators/papermill_operator.py | 0 airflow/operators/pig_operator.py | 0 airflow/operators/postgres_operator.py | 0 airflow/operators/presto_check_operator.py | 0 airflow/operators/presto_to_mysql.py | 0 airflow/operators/python.py | 0 airflow/operators/python_operator.py | 0 airflow/operators/redshift_to_s3_operator.py | 0 airflow/operators/s3_file_transform_operator.py | 0 airflow/operators/s3_to_hive_operator.py | 0 airflow/operators/s3_to_redshift_operator.py | 0 airflow/operators/slack_operator.py | 0 airflow/operators/sql.py | 0 airflow/operators/sql_branch_operator.py | 0 airflow/operators/sqlite_operator.py | 0 airflow/operators/subdag_operator.py | 0 airflow/plugins_manager.py | 0 airflow/providers/__init__.py | 0 .../amazon/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/amazon/README.md | 0 airflow/providers/amazon/__init__.py | 0 airflow/providers/amazon/aws/__init__.py | 0 .../amazon/aws/example_dags/__init__.py | 0 .../aws/example_dags/example_datasync_1.py | 0 .../aws/example_dags/example_datasync_2.py | 0 .../aws/example_dags/example_ecs_fargate.py | 0 .../example_emr_job_flow_automatic_steps.py | 0 .../example_emr_job_flow_manual_steps.py | 0 ...xample_google_api_to_s3_transfer_advanced.py | 0 .../example_google_api_to_s3_transfer_basic.py | 0 .../example_imap_attachment_to_s3.py | 0 .../aws/example_dags/example_s3_bucket.py | 0 .../aws/example_dags/example_s3_to_redshift.py | 0 airflow/providers/amazon/aws/hooks/__init__.py | 0 airflow/providers/amazon/aws/hooks/athena.py | 0 .../providers/amazon/aws/hooks/aws_dynamodb.py | 0 airflow/providers/amazon/aws/hooks/base_aws.py | 0 .../providers/amazon/aws/hooks/batch_client.py | 0 .../amazon/aws/hooks/batch_waiters.json | 0 .../providers/amazon/aws/hooks/batch_waiters.py | 0 .../amazon/aws/hooks/cloud_formation.py | 0 airflow/providers/amazon/aws/hooks/datasync.py | 0 airflow/providers/amazon/aws/hooks/ec2.py | 0 airflow/providers/amazon/aws/hooks/emr.py | 0 airflow/providers/amazon/aws/hooks/glue.py | 0 .../providers/amazon/aws/hooks/glue_catalog.py | 0 airflow/providers/amazon/aws/hooks/kinesis.py | 0 .../amazon/aws/hooks/lambda_function.py | 0 airflow/providers/amazon/aws/hooks/logs.py | 0 airflow/providers/amazon/aws/hooks/redshift.py | 0 airflow/providers/amazon/aws/hooks/s3.py | 0 airflow/providers/amazon/aws/hooks/sagemaker.py | 0 airflow/providers/amazon/aws/hooks/ses.py | 0 airflow/providers/amazon/aws/hooks/sns.py | 0 airflow/providers/amazon/aws/hooks/sqs.py | 0 .../providers/amazon/aws/hooks/step_function.py | 0 airflow/providers/amazon/aws/log/__init__.py | 0 .../amazon/aws/log/cloudwatch_task_handler.py | 0 .../providers/amazon/aws/log/s3_task_handler.py | 0 .../providers/amazon/aws/operators/__init__.py | 0 .../providers/amazon/aws/operators/athena.py | 0 airflow/providers/amazon/aws/operators/batch.py | 0 .../amazon/aws/operators/cloud_formation.py | 0 .../providers/amazon/aws/operators/datasync.py | 0 .../amazon/aws/operators/ec2_start_instance.py | 0 .../amazon/aws/operators/ec2_stop_instance.py | 0 airflow/providers/amazon/aws/operators/ecs.py | 0 .../amazon/aws/operators/emr_add_steps.py | 0 .../amazon/aws/operators/emr_create_job_flow.py | 0 .../amazon/aws/operators/emr_modify_cluster.py | 0 .../aws/operators/emr_terminate_job_flow.py | 0 airflow/providers/amazon/aws/operators/glue.py | 0 .../providers/amazon/aws/operators/s3_bucket.py | 0 .../amazon/aws/operators/s3_copy_object.py | 0 .../amazon/aws/operators/s3_delete_objects.py | 0 .../amazon/aws/operators/s3_file_transform.py | 0 .../providers/amazon/aws/operators/s3_list.py | 0 .../amazon/aws/operators/sagemaker_base.py | 0 .../amazon/aws/operators/sagemaker_endpoint.py | 0 .../aws/operators/sagemaker_endpoint_config.py | 0 .../amazon/aws/operators/sagemaker_model.py | 0 .../aws/operators/sagemaker_processing.py | 0 .../amazon/aws/operators/sagemaker_training.py | 0 .../amazon/aws/operators/sagemaker_transform.py | 0 .../amazon/aws/operators/sagemaker_tuning.py | 0 airflow/providers/amazon/aws/operators/sns.py | 0 airflow/providers/amazon/aws/operators/sqs.py | 0 .../step_function_get_execution_output.py | 0 .../operators/step_function_start_execution.py | 0 .../providers/amazon/aws/secrets/__init__.py | 0 .../amazon/aws/secrets/secrets_manager.py | 0 .../amazon/aws/secrets/systems_manager.py | 0 .../providers/amazon/aws/sensors/__init__.py | 0 airflow/providers/amazon/aws/sensors/athena.py | 0 .../amazon/aws/sensors/cloud_formation.py | 0 .../amazon/aws/sensors/ec2_instance_state.py | 0 .../providers/amazon/aws/sensors/emr_base.py | 0 .../amazon/aws/sensors/emr_job_flow.py | 0 .../providers/amazon/aws/sensors/emr_step.py | 0 airflow/providers/amazon/aws/sensors/glue.py | 0 .../aws/sensors/glue_catalog_partition.py | 0 .../providers/amazon/aws/sensors/redshift.py | 0 airflow/providers/amazon/aws/sensors/s3_key.py | 0 .../amazon/aws/sensors/s3_keys_unchanged.py | 0 .../providers/amazon/aws/sensors/s3_prefix.py | 0 .../amazon/aws/sensors/sagemaker_base.py | 0 .../amazon/aws/sensors/sagemaker_endpoint.py | 0 .../amazon/aws/sensors/sagemaker_training.py | 0 .../amazon/aws/sensors/sagemaker_transform.py | 0 .../amazon/aws/sensors/sagemaker_tuning.py | 0 airflow/providers/amazon/aws/sensors/sqs.py | 0 .../aws/sensors/step_function_execution.py | 0 .../providers/amazon/aws/transfers/__init__.py | 0 .../amazon/aws/transfers/dynamodb_to_s3.py | 0 .../providers/amazon/aws/transfers/gcs_to_s3.py | 0 .../amazon/aws/transfers/google_api_to_s3.py | 0 .../amazon/aws/transfers/hive_to_dynamodb.py | 0 .../aws/transfers/imap_attachment_to_s3.py | 0 .../amazon/aws/transfers/mongo_to_s3.py | 0 .../amazon/aws/transfers/mysql_to_s3.py | 0 .../amazon/aws/transfers/redshift_to_s3.py | 0 .../amazon/aws/transfers/s3_to_redshift.py | 0 .../amazon/aws/transfers/s3_to_sftp.py | 0 .../amazon/aws/transfers/sftp_to_s3.py | 0 airflow/providers/apache/__init__.py | 0 .../cassandra/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/apache/cassandra/README.md | 0 airflow/providers/apache/cassandra/__init__.py | 0 .../apache/cassandra/example_dags/__init__.py | 0 .../example_dags/example_cassandra_dag.py | 0 .../apache/cassandra/hooks/__init__.py | 0 .../apache/cassandra/hooks/cassandra.py | 0 .../apache/cassandra/sensors/__init__.py | 0 .../apache/cassandra/sensors/record.py | 0 .../providers/apache/cassandra/sensors/table.py | 0 .../druid/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/apache/druid/README.md | 0 airflow/providers/apache/druid/__init__.py | 0 .../providers/apache/druid/hooks/__init__.py | 0 airflow/providers/apache/druid/hooks/druid.py | 0 .../apache/druid/operators/__init__.py | 0 .../providers/apache/druid/operators/druid.py | 0 .../apache/druid/operators/druid_check.py | 0 .../apache/druid/transfers/__init__.py | 0 .../apache/druid/transfers/hive_to_druid.py | 0 .../apache/hdfs/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/apache/hdfs/README.md | 0 airflow/providers/apache/hdfs/__init__.py | 0 airflow/providers/apache/hdfs/hooks/__init__.py | 0 airflow/providers/apache/hdfs/hooks/hdfs.py | 0 airflow/providers/apache/hdfs/hooks/webhdfs.py | 0 .../providers/apache/hdfs/sensors/__init__.py | 0 airflow/providers/apache/hdfs/sensors/hdfs.py | 0 .../providers/apache/hdfs/sensors/web_hdfs.py | 0 .../apache/hive/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/apache/hive/README.md | 0 airflow/providers/apache/hive/__init__.py | 0 .../apache/hive/example_dags/__init__.py | 0 .../hive/example_dags/example_twitter_README.md | 0 .../hive/example_dags/example_twitter_dag.py | 0 airflow/providers/apache/hive/hooks/__init__.py | 0 airflow/providers/apache/hive/hooks/hive.py | 0 .../providers/apache/hive/operators/__init__.py | 0 airflow/providers/apache/hive/operators/hive.py | 0 .../apache/hive/operators/hive_stats.py | 0 .../providers/apache/hive/sensors/__init__.py | 0 .../apache/hive/sensors/hive_partition.py | 0 .../apache/hive/sensors/metastore_partition.py | 0 .../apache/hive/sensors/named_hive_partition.py | 0 .../providers/apache/hive/transfers/__init__.py | 0 .../apache/hive/transfers/hive_to_mysql.py | 0 .../apache/hive/transfers/hive_to_samba.py | 0 .../apache/hive/transfers/mssql_to_hive.py | 0 .../apache/hive/transfers/mysql_to_hive.py | 0 .../apache/hive/transfers/s3_to_hive.py | 0 .../apache/hive/transfers/vertica_to_hive.py | 0 airflow/providers/apache/kafka/__init__.py | 0 .../apache/kafka/example_dags/__init__.py | 0 .../kafka/example_dags/example_kafka_dag.py | 0 .../providers/apache/kafka/hooks/__init__.py | 0 .../apache/kafka/hooks/kafka_consumer_hook.py | 0 .../apache/kafka/hooks/kafka_producer_hook.py | 0 .../providers/apache/kafka/sensors/__init__.py | 0 .../apache/kafka/sensors/kafka_sensor.py | 0 airflow/providers/apache/kylin/__init__.py | 0 .../apache/kylin/example_dags/__init__.py | 0 .../kylin/example_dags/example_kylin_dag.py | 0 .../providers/apache/kylin/hooks/__init__.py | 0 airflow/providers/apache/kylin/hooks/kylin.py | 0 .../apache/kylin/operators/__init__.py | 0 .../apache/kylin/operators/kylin_cube.py | 0 .../apache/livy/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/apache/livy/README.md | 0 airflow/providers/apache/livy/__init__.py | 0 .../apache/livy/example_dags/__init__.py | 0 .../apache/livy/example_dags/example_livy.py | 0 airflow/providers/apache/livy/hooks/__init__.py | 0 airflow/providers/apache/livy/hooks/livy.py | 0 .../providers/apache/livy/operators/__init__.py | 0 airflow/providers/apache/livy/operators/livy.py | 0 .../providers/apache/livy/sensors/__init__.py | 0 airflow/providers/apache/livy/sensors/livy.py | 0 .../apache/pig/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/apache/pig/README.md | 0 airflow/providers/apache/pig/__init__.py | 0 .../apache/pig/example_dags/__init__.py | 0 .../apache/pig/example_dags/example_pig.py | 0 airflow/providers/apache/pig/hooks/__init__.py | 0 airflow/providers/apache/pig/hooks/pig.py | 0 .../providers/apache/pig/operators/__init__.py | 0 airflow/providers/apache/pig/operators/pig.py | 0 .../pinot/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/apache/pinot/README.md | 0 airflow/providers/apache/pinot/__init__.py | 0 .../providers/apache/pinot/hooks/__init__.py | 0 airflow/providers/apache/pinot/hooks/pinot.py | 0 .../spark/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/apache/spark/README.md | 0 airflow/providers/apache/spark/__init__.py | 0 .../apache/spark/example_dags/__init__.py | 0 .../spark/example_dags/example_spark_dag.py | 0 .../providers/apache/spark/hooks/__init__.py | 0 .../providers/apache/spark/hooks/spark_jdbc.py | 0 .../apache/spark/hooks/spark_jdbc_script.py | 0 .../providers/apache/spark/hooks/spark_sql.py | 0 .../apache/spark/hooks/spark_submit.py | 0 .../apache/spark/operators/__init__.py | 0 .../apache/spark/operators/spark_jdbc.py | 0 .../apache/spark/operators/spark_sql.py | 0 .../apache/spark/operators/spark_submit.py | 0 .../sqoop/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/apache/sqoop/README.md | 0 airflow/providers/apache/sqoop/__init__.py | 0 .../providers/apache/sqoop/hooks/__init__.py | 0 airflow/providers/apache/sqoop/hooks/sqoop.py | 0 .../apache/sqoop/operators/__init__.py | 0 .../providers/apache/sqoop/operators/sqoop.py | 0 .../celery/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/celery/README.md | 0 airflow/providers/celery/__init__.py | 0 airflow/providers/celery/sensors/__init__.py | 0 .../providers/celery/sensors/celery_queue.py | 0 .../cloudant/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/cloudant/README.md | 0 airflow/providers/cloudant/__init__.py | 0 airflow/providers/cloudant/hooks/__init__.py | 0 airflow/providers/cloudant/hooks/cloudant.py | 0 airflow/providers/cncf/__init__.py | 0 airflow/providers/cncf/kubernetes/__init__.py | 0 .../cncf/kubernetes/example_dags/__init__.py | 0 .../example_dags/example_kubernetes.py | 0 .../example_dags/example_spark_kubernetes.py | 0 .../example_spark_kubernetes_spark_pi.yaml | 0 .../providers/cncf/kubernetes/hooks/__init__.py | 0 .../cncf/kubernetes/hooks/kubernetes.py | 0 .../cncf/kubernetes/operators/__init__.py | 0 .../cncf/kubernetes/operators/kubernetes_pod.py | 0 .../kubernetes/operators/spark_kubernetes.py | 0 .../cncf/kubernetes/sensors/__init__.py | 0 .../cncf/kubernetes/sensors/spark_kubernetes.py | 0 .../databricks/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/databricks/README.md | 0 airflow/providers/databricks/__init__.py | 0 .../databricks/example_dags/__init__.py | 0 .../example_dags/example_databricks.py | 0 airflow/providers/databricks/hooks/__init__.py | 0 .../providers/databricks/hooks/databricks.py | 0 .../providers/databricks/operators/__init__.py | 0 .../databricks/operators/databricks.py | 0 .../datadog/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/datadog/README.md | 0 airflow/providers/datadog/__init__.py | 0 airflow/providers/datadog/hooks/__init__.py | 0 airflow/providers/datadog/hooks/datadog.py | 0 airflow/providers/datadog/sensors/__init__.py | 0 airflow/providers/datadog/sensors/datadog.py | 0 airflow/providers/dependencies.json | 0 .../dingding/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/dingding/README.md | 0 airflow/providers/dingding/__init__.py | 0 .../providers/dingding/example_dags/__init__.py | 0 .../dingding/example_dags/example_dingding.py | 0 airflow/providers/dingding/hooks/__init__.py | 0 airflow/providers/dingding/hooks/dingding.py | 0 .../providers/dingding/operators/__init__.py | 0 .../providers/dingding/operators/dingding.py | 0 .../discord/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/discord/README.md | 0 airflow/providers/discord/__init__.py | 0 airflow/providers/discord/hooks/__init__.py | 0 .../providers/discord/hooks/discord_webhook.py | 0 airflow/providers/discord/operators/__init__.py | 0 .../discord/operators/discord_webhook.py | 0 .../docker/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/docker/README.md | 0 airflow/providers/docker/__init__.py | 0 .../providers/docker/example_dags/__init__.py | 0 .../docker/example_dags/example_docker.py | 0 .../example_dags/example_docker_copy_data.py | 0 .../docker/example_dags/example_docker_swarm.py | 0 airflow/providers/docker/hooks/__init__.py | 0 airflow/providers/docker/hooks/docker.py | 0 airflow/providers/docker/operators/__init__.py | 0 airflow/providers/docker/operators/docker.py | 0 .../providers/docker/operators/docker_swarm.py | 0 .../PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/elasticsearch/README.md | 0 airflow/providers/elasticsearch/__init__.py | 0 .../providers/elasticsearch/hooks/__init__.py | 0 .../elasticsearch/hooks/elasticsearch.py | 0 airflow/providers/elasticsearch/log/__init__.py | 0 .../elasticsearch/log/es_task_handler.py | 0 .../exasol/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/exasol/README.md | 0 airflow/providers/exasol/__init__.py | 0 airflow/providers/exasol/hooks/__init__.py | 0 airflow/providers/exasol/hooks/exasol.py | 0 airflow/providers/exasol/operators/__init__.py | 0 airflow/providers/exasol/operators/exasol.py | 0 .../facebook/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/facebook/README.md | 0 airflow/providers/facebook/__init__.py | 0 airflow/providers/facebook/ads/__init__.py | 0 .../providers/facebook/ads/hooks/__init__.py | 0 airflow/providers/facebook/ads/hooks/ads.py | 0 .../ftp/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/ftp/README.md | 0 airflow/providers/ftp/__init__.py | 0 airflow/providers/ftp/hooks/__init__.py | 0 airflow/providers/ftp/hooks/ftp.py | 0 airflow/providers/ftp/sensors/__init__.py | 0 airflow/providers/ftp/sensors/ftp.py | 0 .../google/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/google/README.md | 0 airflow/providers/google/__init__.py | 0 airflow/providers/google/ads/__init__.py | 0 .../google/ads/example_dags/__init__.py | 0 .../google/ads/example_dags/example_ads.py | 0 airflow/providers/google/ads/hooks/__init__.py | 0 airflow/providers/google/ads/hooks/ads.py | 0 .../providers/google/ads/operators/__init__.py | 0 airflow/providers/google/ads/operators/ads.py | 0 .../providers/google/ads/transfers/__init__.py | 0 .../google/ads/transfers/ads_to_gcs.py | 0 airflow/providers/google/cloud/__init__.py | 0 .../google/cloud/_internal_client/__init__.py | 0 .../_internal_client/secret_manager_client.py | 0 .../google/cloud/example_dags/__init__.py | 0 .../example_automl_nl_text_classification.py | 0 .../example_automl_nl_text_extraction.py | 0 .../example_automl_nl_text_sentiment.py | 0 .../cloud/example_dags/example_automl_tables.py | 0 .../example_dags/example_automl_translation.py | 0 ..._automl_video_intelligence_classification.py | 0 ...xample_automl_video_intelligence_tracking.py | 0 .../example_automl_vision_classification.py | 0 .../example_automl_vision_object_detection.py | 0 .../cloud/example_dags/example_bigquery_dts.py | 0 .../example_dags/example_bigquery_operations.py | 0 .../example_dags/example_bigquery_queries.py | 0 .../example_dags/example_bigquery_query.sql | 0 .../example_dags/example_bigquery_sensors.py | 0 .../example_bigquery_to_bigquery.py | 0 .../example_dags/example_bigquery_to_gcs.py | 0 .../example_dags/example_bigquery_transfer.py | 0 .../cloud/example_dags/example_bigtable.py | 0 .../cloud/example_dags/example_cloud_build.py | 0 .../cloud/example_dags/example_cloud_build.yaml | 0 .../example_dags/example_cloud_memorystore.py | 0 .../cloud/example_dags/example_cloud_sql.py | 0 .../example_dags/example_cloud_sql_query.py | 0 ...xample_cloud_storage_transfer_service_aws.py | 0 ...xample_cloud_storage_transfer_service_gcp.py | 0 .../cloud/example_dags/example_compute.py | 0 .../cloud/example_dags/example_compute_igm.py | 0 .../cloud/example_dags/example_datacatalog.py | 0 .../cloud/example_dags/example_dataflow.py | 0 .../cloud/example_dags/example_datafusion.py | 0 .../cloud/example_dags/example_dataprep.py | 0 .../cloud/example_dags/example_dataproc.py | 0 .../cloud/example_dags/example_datastore.py | 0 .../google/cloud/example_dags/example_dlp.py | 0 .../example_dags/example_facebook_ads_to_gcs.py | 0 .../cloud/example_dags/example_functions.py | 0 .../google/cloud/example_dags/example_gcs.py | 0 .../example_dags/example_gcs_to_bigquery.py | 0 .../cloud/example_dags/example_gcs_to_gcs.py | 0 .../cloud/example_dags/example_gcs_to_sftp.py | 0 .../example_dags/example_kubernetes_engine.py | 0 .../cloud/example_dags/example_life_sciences.py | 0 .../cloud/example_dags/example_local_to_gcs.py | 0 .../cloud/example_dags/example_mlengine.py | 0 .../example_dags/example_natural_language.py | 0 .../example_dags/example_postgres_to_gcs.py | 0 .../cloud/example_dags/example_presto_to_gcs.py | 0 .../google/cloud/example_dags/example_pubsub.py | 0 .../cloud/example_dags/example_sftp_to_gcs.py | 0 .../cloud/example_dags/example_sheets_to_gcs.py | 0 .../cloud/example_dags/example_spanner.py | 0 .../cloud/example_dags/example_spanner.sql | 0 .../example_dags/example_speech_to_text.py | 0 .../cloud/example_dags/example_stackdriver.py | 0 .../google/cloud/example_dags/example_tasks.py | 0 .../example_dags/example_text_to_speech.py | 0 .../cloud/example_dags/example_translate.py | 0 .../example_dags/example_translate_speech.py | 0 .../example_dags/example_video_intelligence.py | 0 .../google/cloud/example_dags/example_vision.py | 0 .../providers/google/cloud/hooks/__init__.py | 0 airflow/providers/google/cloud/hooks/automl.py | 0 .../providers/google/cloud/hooks/bigquery.py | 0 .../google/cloud/hooks/bigquery_dts.py | 0 .../providers/google/cloud/hooks/bigtable.py | 0 .../providers/google/cloud/hooks/cloud_build.py | 0 .../google/cloud/hooks/cloud_memorystore.py | 0 .../providers/google/cloud/hooks/cloud_sql.py | 0 .../hooks/cloud_storage_transfer_service.py | 0 airflow/providers/google/cloud/hooks/compute.py | 0 .../providers/google/cloud/hooks/datacatalog.py | 0 .../providers/google/cloud/hooks/dataflow.py | 0 .../providers/google/cloud/hooks/datafusion.py | 0 .../providers/google/cloud/hooks/dataprep.py | 0 .../providers/google/cloud/hooks/dataproc.py | 0 .../providers/google/cloud/hooks/datastore.py | 0 airflow/providers/google/cloud/hooks/dlp.py | 0 .../providers/google/cloud/hooks/functions.py | 0 airflow/providers/google/cloud/hooks/gcs.py | 0 airflow/providers/google/cloud/hooks/gdm.py | 0 airflow/providers/google/cloud/hooks/kms.py | 0 .../google/cloud/hooks/kubernetes_engine.py | 0 .../google/cloud/hooks/life_sciences.py | 0 .../providers/google/cloud/hooks/mlengine.py | 0 .../google/cloud/hooks/natural_language.py | 0 airflow/providers/google/cloud/hooks/pubsub.py | 0 .../google/cloud/hooks/secret_manager.py | 0 airflow/providers/google/cloud/hooks/spanner.py | 0 .../google/cloud/hooks/speech_to_text.py | 0 .../providers/google/cloud/hooks/stackdriver.py | 0 airflow/providers/google/cloud/hooks/tasks.py | 0 .../google/cloud/hooks/text_to_speech.py | 0 .../providers/google/cloud/hooks/translate.py | 0 .../google/cloud/hooks/video_intelligence.py | 0 airflow/providers/google/cloud/hooks/vision.py | 0 airflow/providers/google/cloud/log/__init__.py | 0 .../google/cloud/log/gcs_task_handler.py | 0 .../cloud/log/stackdriver_task_handler.py | 0 .../google/cloud/operators/__init__.py | 0 .../providers/google/cloud/operators/automl.py | 0 .../google/cloud/operators/bigquery.py | 0 .../google/cloud/operators/bigquery_dts.py | 0 .../google/cloud/operators/bigtable.py | 0 .../google/cloud/operators/cloud_build.py | 0 .../google/cloud/operators/cloud_memorystore.py | 0 .../google/cloud/operators/cloud_sql.py | 0 .../operators/cloud_storage_transfer_service.py | 0 .../providers/google/cloud/operators/compute.py | 0 .../google/cloud/operators/datacatalog.py | 0 .../google/cloud/operators/dataflow.py | 0 .../google/cloud/operators/datafusion.py | 0 .../google/cloud/operators/dataprep.py | 0 .../google/cloud/operators/dataproc.py | 0 .../google/cloud/operators/datastore.py | 0 airflow/providers/google/cloud/operators/dlp.py | 0 .../google/cloud/operators/functions.py | 0 airflow/providers/google/cloud/operators/gcs.py | 0 .../google/cloud/operators/kubernetes_engine.py | 0 .../google/cloud/operators/life_sciences.py | 0 .../google/cloud/operators/mlengine.py | 0 .../google/cloud/operators/natural_language.py | 0 .../providers/google/cloud/operators/pubsub.py | 0 .../providers/google/cloud/operators/spanner.py | 0 .../google/cloud/operators/speech_to_text.py | 0 .../google/cloud/operators/stackdriver.py | 0 .../providers/google/cloud/operators/tasks.py | 0 .../google/cloud/operators/text_to_speech.py | 0 .../google/cloud/operators/translate.py | 0 .../google/cloud/operators/translate_speech.py | 0 .../cloud/operators/video_intelligence.py | 0 .../providers/google/cloud/operators/vision.py | 0 .../providers/google/cloud/secrets/__init__.py | 0 .../google/cloud/secrets/secret_manager.py | 0 .../providers/google/cloud/sensors/__init__.py | 0 .../providers/google/cloud/sensors/bigquery.py | 0 .../google/cloud/sensors/bigquery_dts.py | 0 .../providers/google/cloud/sensors/bigtable.py | 0 .../sensors/cloud_storage_transfer_service.py | 0 airflow/providers/google/cloud/sensors/gcs.py | 0 .../providers/google/cloud/sensors/pubsub.py | 0 .../google/cloud/transfers/__init__.py | 0 .../google/cloud/transfers/adls_to_gcs.py | 0 .../cloud/transfers/bigquery_to_bigquery.py | 0 .../google/cloud/transfers/bigquery_to_gcs.py | 0 .../google/cloud/transfers/bigquery_to_mysql.py | 0 .../google/cloud/transfers/cassandra_to_gcs.py | 0 .../cloud/transfers/facebook_ads_to_gcs.py | 0 .../google/cloud/transfers/gcs_to_bigquery.py | 0 .../google/cloud/transfers/gcs_to_gcs.py | 0 .../google/cloud/transfers/gcs_to_local.py | 0 .../google/cloud/transfers/gcs_to_sftp.py | 0 .../google/cloud/transfers/local_to_gcs.py | 0 .../google/cloud/transfers/mssql_to_gcs.py | 0 .../google/cloud/transfers/mysql_to_gcs.py | 0 .../google/cloud/transfers/postgres_to_gcs.py | 0 .../google/cloud/transfers/presto_to_gcs.py | 0 .../google/cloud/transfers/s3_to_gcs.py | 0 .../google/cloud/transfers/sftp_to_gcs.py | 0 .../google/cloud/transfers/sheets_to_gcs.py | 0 .../google/cloud/transfers/sql_to_gcs.py | 0 .../providers/google/cloud/utils/__init__.py | 0 .../google/cloud/utils/credentials_provider.py | 0 .../google/cloud/utils/field_sanitizer.py | 0 .../google/cloud/utils/field_validator.py | 0 .../cloud/utils/mlengine_operator_utils.py | 0 .../cloud/utils/mlengine_prediction_summary.py | 0 airflow/providers/google/common/__init__.py | 0 .../google/common/auth_backend/__init__.py | 0 .../google/common/auth_backend/google_openid.py | 0 .../providers/google/common/hooks/__init__.py | 0 .../google/common/hooks/base_google.py | 0 .../google/common/hooks/discovery_api.py | 0 .../providers/google/common/utils/__init__.py | 0 .../google/common/utils/id_token_credentials.py | 0 .../google/config_templates/config.yml | 0 .../google/config_templates/default_config.cfg | 0 airflow/providers/google/firebase/__init__.py | 0 .../google/firebase/example_dags/__init__.py | 0 .../firebase/example_dags/example_firestore.py | 0 .../providers/google/firebase/hooks/__init__.py | 0 .../google/firebase/hooks/firestore.py | 0 .../google/firebase/operators/__init__.py | 0 .../google/firebase/operators/firestore.py | 0 .../google/marketing_platform/__init__.py | 0 .../marketing_platform/example_dags/__init__.py | 0 .../example_dags/example_analytics.py | 0 .../example_dags/example_campaign_manager.py | 0 .../example_dags/example_display_video.py | 0 .../example_dags/example_search_ads.py | 0 .../google/marketing_platform/hooks/__init__.py | 0 .../marketing_platform/hooks/analytics.py | 0 .../hooks/campaign_manager.py | 0 .../marketing_platform/hooks/display_video.py | 0 .../marketing_platform/hooks/search_ads.py | 0 .../marketing_platform/operators/__init__.py | 0 .../marketing_platform/operators/analytics.py | 0 .../operators/campaign_manager.py | 0 .../operators/display_video.py | 0 .../marketing_platform/operators/search_ads.py | 0 .../marketing_platform/sensors/__init__.py | 0 .../sensors/campaign_manager.py | 0 .../marketing_platform/sensors/display_video.py | 0 .../marketing_platform/sensors/search_ads.py | 0 airflow/providers/google/suite/__init__.py | 0 .../google/suite/example_dags/__init__.py | 0 .../suite/example_dags/example_gcs_to_gdrive.py | 0 .../suite/example_dags/example_gcs_to_sheets.py | 0 .../google/suite/example_dags/example_sheets.py | 0 .../providers/google/suite/hooks/__init__.py | 0 airflow/providers/google/suite/hooks/drive.py | 0 airflow/providers/google/suite/hooks/sheets.py | 0 .../google/suite/operators/__init__.py | 0 .../providers/google/suite/operators/sheets.py | 0 .../google/suite/transfers/__init__.py | 0 .../google/suite/transfers/gcs_to_gdrive.py | 0 .../google/suite/transfers/gcs_to_sheets.py | 0 .../grpc/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/grpc/README.md | 0 airflow/providers/grpc/__init__.py | 0 airflow/providers/grpc/hooks/__init__.py | 0 airflow/providers/grpc/hooks/grpc.py | 0 airflow/providers/grpc/operators/__init__.py | 0 airflow/providers/grpc/operators/grpc.py | 0 .../hashicorp/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/hashicorp/README.md | 0 airflow/providers/hashicorp/__init__.py | 0 .../hashicorp/_internal_client/__init__.py | 0 .../hashicorp/_internal_client/vault_client.py | 0 airflow/providers/hashicorp/hooks/__init__.py | 0 airflow/providers/hashicorp/hooks/vault.py | 0 airflow/providers/hashicorp/secrets/__init__.py | 0 airflow/providers/hashicorp/secrets/vault.py | 0 .../http/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/http/README.md | 0 airflow/providers/http/__init__.py | 0 airflow/providers/http/example_dags/__init__.py | 0 .../providers/http/example_dags/example_http.py | 0 airflow/providers/http/hooks/__init__.py | 0 airflow/providers/http/hooks/http.py | 0 airflow/providers/http/operators/__init__.py | 0 airflow/providers/http/operators/http.py | 0 airflow/providers/http/sensors/__init__.py | 0 airflow/providers/http/sensors/http.py | 0 .../imap/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/imap/README.md | 0 airflow/providers/imap/__init__.py | 0 airflow/providers/imap/hooks/__init__.py | 0 airflow/providers/imap/hooks/imap.py | 0 airflow/providers/imap/sensors/__init__.py | 0 .../providers/imap/sensors/imap_attachment.py | 0 .../jdbc/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/jdbc/README.md | 0 airflow/providers/jdbc/__init__.py | 0 airflow/providers/jdbc/hooks/__init__.py | 0 airflow/providers/jdbc/hooks/jdbc.py | 0 airflow/providers/jdbc/operators/__init__.py | 0 airflow/providers/jdbc/operators/jdbc.py | 0 .../jenkins/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/jenkins/README.md | 0 airflow/providers/jenkins/__init__.py | 0 .../providers/jenkins/example_dags/__init__.py | 0 .../example_dags/example_jenkins_job_trigger.py | 0 airflow/providers/jenkins/hooks/__init__.py | 0 airflow/providers/jenkins/hooks/jenkins.py | 0 airflow/providers/jenkins/operators/__init__.py | 0 .../jenkins/operators/jenkins_job_trigger.py | 0 .../jira/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/jira/README.md | 0 airflow/providers/jira/__init__.py | 0 airflow/providers/jira/hooks/__init__.py | 0 airflow/providers/jira/hooks/jira.py | 0 airflow/providers/jira/operators/__init__.py | 0 airflow/providers/jira/operators/jira.py | 0 airflow/providers/jira/sensors/__init__.py | 0 airflow/providers/jira/sensors/jira.py | 0 airflow/providers/microsoft/__init__.py | 0 .../azure/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/microsoft/azure/README.md | 0 airflow/providers/microsoft/azure/__init__.py | 0 .../microsoft/azure/example_dags/__init__.py | 0 .../example_azure_container_instances.py | 0 .../example_dags/example_azure_cosmosdb.py | 0 .../providers/microsoft/azure/hooks/__init__.py | 0 airflow/providers/microsoft/azure/hooks/adx.py | 0 .../microsoft/azure/hooks/azure_batch.py | 0 .../azure/hooks/azure_container_instance.py | 0 .../azure/hooks/azure_container_registry.py | 0 .../azure/hooks/azure_container_volume.py | 0 .../microsoft/azure/hooks/azure_cosmos.py | 0 .../microsoft/azure/hooks/azure_data_lake.py | 0 .../microsoft/azure/hooks/azure_fileshare.py | 0 .../microsoft/azure/hooks/base_azure.py | 0 airflow/providers/microsoft/azure/hooks/wasb.py | 0 .../providers/microsoft/azure/log/__init__.py | 0 .../microsoft/azure/log/wasb_task_handler.py | 0 .../microsoft/azure/operators/__init__.py | 0 .../microsoft/azure/operators/adls_list.py | 0 .../providers/microsoft/azure/operators/adx.py | 0 .../microsoft/azure/operators/azure_batch.py | 0 .../operators/azure_container_instances.py | 0 .../microsoft/azure/operators/azure_cosmos.py | 0 .../azure/operators/wasb_delete_blob.py | 0 .../microsoft/azure/sensors/__init__.py | 0 .../microsoft/azure/sensors/azure_cosmos.py | 0 .../providers/microsoft/azure/sensors/wasb.py | 0 .../microsoft/azure/transfers/__init__.py | 0 .../microsoft/azure/transfers/file_to_wasb.py | 0 .../transfers/oracle_to_azure_data_lake.py | 0 .../mssql/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/microsoft/mssql/README.md | 0 airflow/providers/microsoft/mssql/__init__.py | 0 .../providers/microsoft/mssql/hooks/__init__.py | 0 .../providers/microsoft/mssql/hooks/mssql.py | 0 .../microsoft/mssql/operators/__init__.py | 0 .../microsoft/mssql/operators/mssql.py | 0 .../winrm/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/microsoft/winrm/README.md | 0 airflow/providers/microsoft/winrm/__init__.py | 0 .../microsoft/winrm/example_dags/__init__.py | 0 .../winrm/example_dags/example_winrm.py | 0 .../providers/microsoft/winrm/hooks/__init__.py | 0 .../providers/microsoft/winrm/hooks/winrm.py | 0 .../microsoft/winrm/operators/__init__.py | 0 .../microsoft/winrm/operators/winrm.py | 0 .../mongo/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/mongo/README.md | 0 airflow/providers/mongo/__init__.py | 0 airflow/providers/mongo/hooks/__init__.py | 0 airflow/providers/mongo/hooks/mongo.py | 0 airflow/providers/mongo/sensors/__init__.py | 0 airflow/providers/mongo/sensors/mongo.py | 0 .../mysql/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/mysql/README.md | 0 airflow/providers/mysql/__init__.py | 0 airflow/providers/mysql/hooks/__init__.py | 0 airflow/providers/mysql/hooks/mysql.py | 0 airflow/providers/mysql/operators/__init__.py | 0 airflow/providers/mysql/operators/mysql.py | 0 airflow/providers/mysql/transfers/__init__.py | 0 .../mysql/transfers/presto_to_mysql.py | 0 .../providers/mysql/transfers/s3_to_mysql.py | 0 .../mysql/transfers/vertica_to_mysql.py | 0 .../odbc/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/odbc/README.md | 0 airflow/providers/odbc/__init__.py | 0 airflow/providers/odbc/hooks/__init__.py | 0 airflow/providers/odbc/hooks/odbc.py | 0 .../openfaas/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/openfaas/README.md | 0 airflow/providers/openfaas/__init__.py | 0 airflow/providers/openfaas/hooks/__init__.py | 0 airflow/providers/openfaas/hooks/openfaas.py | 0 .../opsgenie/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/opsgenie/README.md | 0 airflow/providers/opsgenie/__init__.py | 0 airflow/providers/opsgenie/hooks/__init__.py | 0 .../providers/opsgenie/hooks/opsgenie_alert.py | 0 .../providers/opsgenie/operators/__init__.py | 0 .../opsgenie/operators/opsgenie_alert.py | 0 .../oracle/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/oracle/README.md | 0 airflow/providers/oracle/__init__.py | 0 airflow/providers/oracle/hooks/__init__.py | 0 airflow/providers/oracle/hooks/oracle.py | 0 airflow/providers/oracle/operators/__init__.py | 0 airflow/providers/oracle/operators/oracle.py | 0 airflow/providers/oracle/transfers/__init__.py | 0 .../oracle/transfers/oracle_to_oracle.py | 0 .../pagerduty/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/pagerduty/README.md | 0 airflow/providers/pagerduty/__init__.py | 0 airflow/providers/pagerduty/hooks/__init__.py | 0 airflow/providers/pagerduty/hooks/pagerduty.py | 0 airflow/providers/papermill/ADDITIONAL_INFO.md | 0 airflow/providers/papermill/__init__.py | 0 .../papermill/example_dags/__init__.py | 0 .../papermill/example_dags/example_papermill.py | 0 .../providers/papermill/operators/__init__.py | 0 .../providers/papermill/operators/papermill.py | 0 .../postgres/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/postgres/README.md | 0 airflow/providers/postgres/__init__.py | 0 airflow/providers/postgres/hooks/__init__.py | 0 airflow/providers/postgres/hooks/postgres.py | 0 .../providers/postgres/operators/__init__.py | 0 .../providers/postgres/operators/postgres.py | 0 .../presto/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/presto/README.md | 0 airflow/providers/presto/__init__.py | 0 airflow/providers/presto/hooks/__init__.py | 0 airflow/providers/presto/hooks/presto.py | 0 .../qubole/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/qubole/README.md | 0 airflow/providers/qubole/__init__.py | 0 .../providers/qubole/example_dags/__init__.py | 0 .../qubole/example_dags/example_qubole.py | 0 airflow/providers/qubole/hooks/__init__.py | 0 airflow/providers/qubole/hooks/qubole.py | 0 airflow/providers/qubole/hooks/qubole_check.py | 0 airflow/providers/qubole/operators/__init__.py | 0 airflow/providers/qubole/operators/qubole.py | 0 .../providers/qubole/operators/qubole_check.py | 0 airflow/providers/qubole/sensors/__init__.py | 0 airflow/providers/qubole/sensors/qubole.py | 0 .../redis/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/redis/README.md | 0 airflow/providers/redis/__init__.py | 0 airflow/providers/redis/hooks/__init__.py | 0 airflow/providers/redis/hooks/redis.py | 0 airflow/providers/redis/operators/__init__.py | 0 .../providers/redis/operators/redis_publish.py | 0 airflow/providers/redis/sensors/__init__.py | 0 airflow/providers/redis/sensors/redis_key.py | 0 .../providers/redis/sensors/redis_pub_sub.py | 0 .../salesforce/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/salesforce/README.md | 0 airflow/providers/salesforce/__init__.py | 0 .../salesforce/example_dags/__init__.py | 0 .../example_tableau_refresh_workbook.py | 0 airflow/providers/salesforce/hooks/__init__.py | 0 .../providers/salesforce/hooks/salesforce.py | 0 airflow/providers/salesforce/hooks/tableau.py | 0 .../providers/salesforce/operators/__init__.py | 0 .../operators/tableau_refresh_workbook.py | 0 .../providers/salesforce/sensors/__init__.py | 0 .../salesforce/sensors/tableau_job_status.py | 0 .../samba/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/samba/README.md | 0 airflow/providers/samba/__init__.py | 0 airflow/providers/samba/hooks/__init__.py | 0 airflow/providers/samba/hooks/samba.py | 0 .../segment/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/segment/README.md | 0 airflow/providers/segment/__init__.py | 0 airflow/providers/segment/hooks/__init__.py | 0 airflow/providers/segment/hooks/segment.py | 0 airflow/providers/segment/operators/__init__.py | 0 .../segment/operators/segment_track_event.py | 0 airflow/providers/sendgrid/__init__.py | 0 airflow/providers/sendgrid/utils/__init__.py | 0 airflow/providers/sendgrid/utils/emailer.py | 0 .../sftp/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/sftp/README.md | 0 airflow/providers/sftp/__init__.py | 0 airflow/providers/sftp/hooks/__init__.py | 0 airflow/providers/sftp/hooks/sftp.py | 0 airflow/providers/sftp/operators/__init__.py | 0 airflow/providers/sftp/operators/sftp.py | 0 airflow/providers/sftp/sensors/__init__.py | 0 airflow/providers/sftp/sensors/sftp.py | 0 .../singularity/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/singularity/README.md | 0 airflow/providers/singularity/__init__.py | 0 .../singularity/example_dags/__init__.py | 0 .../example_dags/example_singularity.py | 0 .../providers/singularity/operators/__init__.py | 0 .../singularity/operators/singularity.py | 0 .../slack/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/slack/README.md | 0 airflow/providers/slack/__init__.py | 0 airflow/providers/slack/hooks/__init__.py | 0 airflow/providers/slack/hooks/slack.py | 0 airflow/providers/slack/hooks/slack_webhook.py | 0 airflow/providers/slack/operators/__init__.py | 0 airflow/providers/slack/operators/slack.py | 0 .../providers/slack/operators/slack_webhook.py | 0 .../snowflake/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/snowflake/README.md | 0 airflow/providers/snowflake/__init__.py | 0 .../snowflake/example_dags/__init__.py | 0 .../snowflake/example_dags/example_snowflake.py | 0 airflow/providers/snowflake/hooks/__init__.py | 0 airflow/providers/snowflake/hooks/snowflake.py | 0 .../providers/snowflake/operators/__init__.py | 0 .../providers/snowflake/operators/snowflake.py | 0 .../providers/snowflake/transfers/__init__.py | 0 .../snowflake/transfers/s3_to_snowflake.py | 0 .../snowflake/transfers/snowflake_to_slack.py | 0 .../sqlite/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/sqlite/README.md | 0 airflow/providers/sqlite/__init__.py | 0 airflow/providers/sqlite/hooks/__init__.py | 0 airflow/providers/sqlite/hooks/sqlite.py | 0 airflow/providers/sqlite/operators/__init__.py | 0 airflow/providers/sqlite/operators/sqlite.py | 0 .../ssh/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/ssh/README.md | 0 airflow/providers/ssh/__init__.py | 0 airflow/providers/ssh/hooks/__init__.py | 0 airflow/providers/ssh/hooks/ssh.py | 0 airflow/providers/ssh/operators/__init__.py | 0 airflow/providers/ssh/operators/ssh.py | 0 .../vertica/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/vertica/README.md | 0 airflow/providers/vertica/__init__.py | 0 airflow/providers/vertica/hooks/__init__.py | 0 airflow/providers/vertica/hooks/vertica.py | 0 airflow/providers/vertica/operators/__init__.py | 0 airflow/providers/vertica/operators/vertica.py | 0 .../yandex/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/yandex/README.md | 0 airflow/providers/yandex/__init__.py | 0 .../providers/yandex/example_dags/__init__.py | 0 .../example_yandexcloud_dataproc.py | 0 airflow/providers/yandex/hooks/__init__.py | 0 airflow/providers/yandex/hooks/yandex.py | 0 .../yandex/hooks/yandexcloud_dataproc.py | 0 airflow/providers/yandex/operators/__init__.py | 0 .../yandex/operators/yandexcloud_dataproc.py | 0 .../zendesk/PROVIDERS_CHANGES_2020.06.24.md | 0 airflow/providers/zendesk/README.md | 0 airflow/providers/zendesk/__init__.py | 0 airflow/providers/zendesk/hooks/__init__.py | 0 airflow/providers/zendesk/hooks/zendesk.py | 0 airflow/py.typed | 0 airflow/secrets/__init__.py | 0 airflow/secrets/base_secrets.py | 0 airflow/secrets/environment_variables.py | 0 airflow/secrets/local_filesystem.py | 0 airflow/secrets/metastore.py | 0 airflow/security/__init__.py | 0 airflow/security/kerberos.py | 0 airflow/security/utils.py | 0 airflow/sensors/__init__.py | 0 airflow/sensors/base_sensor_operator.py | 0 airflow/sensors/bash.py | 0 airflow/sensors/date_time_sensor.py | 0 airflow/sensors/external_task_sensor.py | 0 airflow/sensors/filesystem.py | 0 airflow/sensors/hdfs_sensor.py | 0 airflow/sensors/hive_partition_sensor.py | 0 airflow/sensors/http_sensor.py | 0 airflow/sensors/metastore_partition_sensor.py | 0 airflow/sensors/named_hive_partition_sensor.py | 0 airflow/sensors/python.py | 0 airflow/sensors/s3_key_sensor.py | 0 airflow/sensors/s3_prefix_sensor.py | 0 airflow/sensors/sql_sensor.py | 0 airflow/sensors/time_delta_sensor.py | 0 airflow/sensors/time_sensor.py | 0 airflow/sensors/web_hdfs_sensor.py | 0 airflow/sensors/weekday_sensor.py | 0 airflow/sentry.py | 0 airflow/serialization/__init__.py | 0 airflow/serialization/enums.py | 0 airflow/serialization/helpers.py | 0 airflow/serialization/json_schema.py | 0 airflow/serialization/schema.json | 0 airflow/serialization/serialized_objects.py | 0 airflow/settings.py | 0 airflow/stats.py | 0 airflow/task/__init__.py | 0 airflow/task/task_runner/__init__.py | 0 airflow/task/task_runner/base_task_runner.py | 0 airflow/task/task_runner/cgroup_task_runner.py | 0 .../task/task_runner/standard_task_runner.py | 0 airflow/ti_deps/__init__.py | 0 airflow/ti_deps/dep_context.py | 0 airflow/ti_deps/dependencies_deps.py | 0 airflow/ti_deps/dependencies_states.py | 0 airflow/ti_deps/deps/__init__.py | 0 airflow/ti_deps/deps/base_ti_dep.py | 0 .../ti_deps/deps/dag_ti_slots_available_dep.py | 0 airflow/ti_deps/deps/dag_unpaused_dep.py | 0 airflow/ti_deps/deps/dagrun_exists_dep.py | 0 airflow/ti_deps/deps/dagrun_id_dep.py | 0 .../deps/exec_date_after_start_date_dep.py | 0 airflow/ti_deps/deps/not_in_retry_period_dep.py | 0 .../ti_deps/deps/not_previously_skipped_dep.py | 0 .../ti_deps/deps/pool_slots_available_dep.py | 0 airflow/ti_deps/deps/prev_dagrun_dep.py | 0 airflow/ti_deps/deps/ready_to_reschedule.py | 0 airflow/ti_deps/deps/runnable_exec_date_dep.py | 0 airflow/ti_deps/deps/task_concurrency_dep.py | 0 airflow/ti_deps/deps/task_not_running_dep.py | 0 airflow/ti_deps/deps/trigger_rule_dep.py | 0 airflow/ti_deps/deps/valid_state_dep.py | 0 airflow/typing_compat.py | 0 airflow/utils/__init__.py | 0 airflow/utils/asciiart.py | 0 airflow/utils/cli.py | 0 airflow/utils/cli_action_loggers.py | 0 airflow/utils/code_utils.py | 0 airflow/utils/compression.py | 0 airflow/utils/configuration.py | 0 airflow/utils/dag_cycle_tester.py | 0 airflow/utils/dag_processing.py | 0 airflow/utils/dates.py | 0 airflow/utils/db.py | 0 airflow/utils/decorators.py | 0 airflow/utils/docs.py | 0 airflow/utils/dot_renderer.py | 0 airflow/utils/email.py | 0 airflow/utils/file.py | 0 airflow/utils/helpers.py | 0 airflow/utils/json.py | 0 airflow/utils/log/__init__.py | 0 airflow/utils/log/cloudwatch_task_handler.py | 0 airflow/utils/log/colored_log.py | 0 airflow/utils/log/es_task_handler.py | 0 airflow/utils/log/file_processor_handler.py | 0 airflow/utils/log/file_task_handler.py | 0 airflow/utils/log/gcs_task_handler.py | 0 airflow/utils/log/json_formatter.py | 0 airflow/utils/log/log_reader.py | 0 airflow/utils/log/logging_mixin.py | 0 airflow/utils/log/s3_task_handler.py | 0 airflow/utils/log/stackdriver_task_handler.py | 0 .../log/task_handler_with_custom_formatter.py | 0 airflow/utils/log/wasb_task_handler.py | 0 airflow/utils/mixins.py | 0 airflow/utils/module_loading.py | 0 airflow/utils/net.py | 0 airflow/utils/operator_helpers.py | 0 airflow/utils/operator_resources.py | 0 airflow/utils/orm_event_handlers.py | 0 airflow/utils/platform.py | 0 airflow/utils/process_utils.py | 0 airflow/utils/python_virtualenv.py | 0 airflow/utils/python_virtualenv_script.jinja2 | 0 airflow/utils/serve_logs.py | 0 airflow/utils/session.py | 0 airflow/utils/sqlalchemy.py | 0 airflow/utils/state.py | 0 airflow/utils/strings.py | 0 airflow/utils/timeout.py | 0 airflow/utils/timezone.py | 0 airflow/utils/trigger_rule.py | 0 airflow/utils/types.py | 0 airflow/utils/weekday.py | 0 airflow/utils/weight_rule.py | 0 airflow/version.py | 0 airflow/www/.eslintignore | 0 airflow/www/.eslintrc | 0 airflow/www/.stylelintignore | 0 airflow/www/.stylelintrc | 0 airflow/www/__init__.py | 0 airflow/www/api/__init__.py | 0 airflow/www/api/experimental/__init__.py | 0 airflow/www/api/experimental/endpoints.py | 0 airflow/www/app.py | 0 airflow/www/blueprints.py | 0 airflow/www/decorators.py | 0 airflow/www/extensions/__init__.py | 0 airflow/www/extensions/init_appbuilder.py | 0 airflow/www/extensions/init_appbuilder_links.py | 0 airflow/www/extensions/init_dagbag.py | 0 airflow/www/extensions/init_jinja_globals.py | 0 airflow/www/extensions/init_manifest_files.py | 0 airflow/www/extensions/init_security.py | 0 airflow/www/extensions/init_session.py | 0 airflow/www/extensions/init_views.py | 0 airflow/www/extensions/init_wsgi_middlewares.py | 0 airflow/www/forms.py | 0 airflow/www/gunicorn_config.py | 0 airflow/www/package.json | 0 airflow/www/security.py | 0 airflow/www/static/airflow.gif | Bin airflow/www/static/css/bootstrap-theme.css | 0 airflow/www/static/css/flash.css | 0 airflow/www/static/css/gantt.css | 0 airflow/www/static/css/graph.css | 0 airflow/www/static/css/main.css | 0 airflow/www/static/css/tree.css | 0 airflow/www/static/js/base.js | 0 airflow/www/static/js/connection_form.js | 0 airflow/www/static/js/datetime-utils.js | 0 airflow/www/static/js/gantt-chart-d3v2.js | 0 airflow/www/static/js/graph.js | 0 airflow/www/static/js/ie.js | 0 airflow/www/static/js/task-instances.js | 0 airflow/www/static/loading.gif | Bin airflow/www/static/pin.svg | 0 airflow/www/static/pin_100.png | Bin airflow/www/static/pin_25.png | Bin airflow/www/static/pin_32.png | Bin airflow/www/static/pin_35.png | Bin airflow/www/static/pin_40.png | Bin airflow/www/static/pin_large.png | Bin airflow/www/static/screenshots/gantt.png | Bin airflow/www/static/screenshots/graph.png | Bin airflow/www/static/screenshots/tree.png | Bin airflow/www/static/sort_asc.png | Bin airflow/www/static/sort_both.png | Bin airflow/www/static/sort_desc.png | Bin airflow/www/templates/airflow/chart.html | 0 airflow/www/templates/airflow/circles.html | 0 airflow/www/templates/airflow/code.html | 0 airflow/www/templates/airflow/config.html | 0 airflow/www/templates/airflow/confirm.html | 0 airflow/www/templates/airflow/conn_create.html | 0 airflow/www/templates/airflow/conn_edit.html | 0 airflow/www/templates/airflow/dag.html | 0 airflow/www/templates/airflow/dag_code.html | 0 airflow/www/templates/airflow/dag_details.html | 0 airflow/www/templates/airflow/dags.html | 0 .../www/templates/airflow/duration_chart.html | 0 airflow/www/templates/airflow/gantt.html | 0 airflow/www/templates/airflow/graph.html | 0 airflow/www/templates/airflow/master.html | 0 airflow/www/templates/airflow/model_list.html | 0 airflow/www/templates/airflow/noaccess.html | 0 airflow/www/templates/airflow/redoc.html | 0 airflow/www/templates/airflow/task.html | 0 .../www/templates/airflow/task_instance.html | 0 airflow/www/templates/airflow/ti_code.html | 0 airflow/www/templates/airflow/ti_log.html | 0 airflow/www/templates/airflow/traceback.html | 0 airflow/www/templates/airflow/tree.html | 0 airflow/www/templates/airflow/trigger.html | 0 .../www/templates/airflow/variable_edit.html | 0 .../www/templates/airflow/variable_list.html | 0 airflow/www/templates/airflow/version.html | 0 airflow/www/templates/airflow/xcom.html | 0 .../templates/analytics/google_analytics.html | 0 airflow/www/templates/analytics/metarouter.html | 0 airflow/www/templates/analytics/segment.html | 0 airflow/www/templates/appbuilder/flash.html | 0 airflow/www/templates/appbuilder/index.html | 0 airflow/www/templates/appbuilder/navbar.html | 0 .../www/templates/appbuilder/navbar_menu.html | 0 .../www/templates/appbuilder/navbar_right.html | 0 airflow/www/utils.py | 0 airflow/www/validators.py | 0 airflow/www/views.py | 0 airflow/www/webpack.config.js | 0 airflow/www/widgets.py | 0 airflow/www/yarn.lock | 0 breeze-complete | 0 codecov.yml | 0 pylintrc | 0 pytest.ini | 0 scripts/ci/docker-compose/local.yml | 16 ++++------------ setup.cfg | 0 setup.py | 0 yamllint-config.yml | 0 1639 files changed, 4 insertions(+), 12 deletions(-) mode change 100644 => 100755 BREEZE.rst mode change 100644 => 100755 CHANGELOG.txt mode change 100644 => 100755 CI.rst mode change 100644 => 100755 CODE_OF_CONDUCT.md mode change 100644 => 100755 CONTRIBUTING.rst mode change 100644 => 100755 Dockerfile mode change 100644 => 100755 Dockerfile.ci mode change 100644 => 100755 IMAGES.rst mode change 100644 => 100755 INSTALL mode change 100644 => 100755 LICENSE mode change 100644 => 100755 LOCAL_VIRTUALENV.rst mode change 100644 => 100755 MANIFEST.in mode change 100644 => 100755 NOTICE mode change 100644 => 100755 README.md mode change 100644 => 100755 STATIC_CODE_CHECKS.rst mode change 100644 => 100755 TESTING.rst mode change 100644 => 100755 UPDATING.md mode change 100644 => 100755 airflow/__init__.py mode change 100644 => 100755 airflow/__main__.py mode change 100644 => 100755 airflow/alembic.ini mode change 100644 => 100755 airflow/api/__init__.py mode change 100644 => 100755 airflow/api/auth/__init__.py mode change 100644 => 100755 airflow/api/auth/backend/__init__.py mode change 100644 => 100755 airflow/api/auth/backend/basic_auth.py mode change 100644 => 100755 airflow/api/auth/backend/default.py mode change 100644 => 100755 airflow/api/auth/backend/deny_all.py mode change 100644 => 100755 airflow/api/auth/backend/kerberos_auth.py mode change 100644 => 100755 airflow/api/client/__init__.py mode change 100644 => 100755 airflow/api/client/api_client.py mode change 100644 => 100755 airflow/api/client/json_client.py mode change 100644 => 100755 airflow/api/client/local_client.py mode change 100644 => 100755 airflow/api/common/__init__.py mode change 100644 => 100755 airflow/api/common/experimental/__init__.py mode change 100644 => 100755 airflow/api/common/experimental/delete_dag.py mode change 100644 => 100755 airflow/api/common/experimental/get_code.py mode change 100644 => 100755 airflow/api/common/experimental/get_dag_run_state.py mode change 100644 => 100755 airflow/api/common/experimental/get_dag_runs.py mode change 100644 => 100755 airflow/api/common/experimental/get_lineage.py mode change 100644 => 100755 airflow/api/common/experimental/get_task.py mode change 100644 => 100755 airflow/api/common/experimental/get_task_instance.py mode change 100644 => 100755 airflow/api/common/experimental/mark_tasks.py mode change 100644 => 100755 airflow/api/common/experimental/pool.py mode change 100644 => 100755 airflow/api/common/experimental/trigger_dag.py mode change 100644 => 100755 airflow/api_connexion/__init__.py mode change 100644 => 100755 airflow/api_connexion/endpoints/__init__.py mode change 100644 => 100755 airflow/api_connexion/endpoints/config_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/connection_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/dag_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/dag_run_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/dag_source_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/event_log_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/extra_link_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/health_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/import_error_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/log_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/pool_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/task_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/task_instance_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/variable_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/version_endpoint.py mode change 100644 => 100755 airflow/api_connexion/endpoints/xcom_endpoint.py mode change 100644 => 100755 airflow/api_connexion/exceptions.py mode change 100644 => 100755 airflow/api_connexion/openapi/v1.yaml mode change 100644 => 100755 airflow/api_connexion/parameters.py mode change 100644 => 100755 airflow/api_connexion/schemas/__init__.py mode change 100644 => 100755 airflow/api_connexion/schemas/common_schema.py mode change 100644 => 100755 airflow/api_connexion/schemas/config_schema.py mode change 100644 => 100755 airflow/api_connexion/schemas/connection_schema.py mode change 100644 => 100755 airflow/api_connexion/schemas/dag_run_schema.py mode change 100644 => 100755 airflow/api_connexion/schemas/dag_schema.py mode change 100644 => 100755 airflow/api_connexion/schemas/dag_source_schema.py mode change 100644 => 100755 airflow/api_connexion/schemas/enum_schemas.py mode change 100644 => 100755 airflow/api_connexion/schemas/error_schema.py mode change 100644 => 100755 airflow/api_connexion/schemas/event_log_schema.py mode change 100644 => 100755 airflow/api_connexion/schemas/health_schema.py mode change 100644 => 100755 airflow/api_connexion/schemas/log_schema.py mode change 100644 => 100755 airflow/api_connexion/schemas/pool_schema.py mode change 100644 => 100755 airflow/api_connexion/schemas/task_schema.py mode change 100644 => 100755 airflow/api_connexion/schemas/variable_schema.py mode change 100644 => 100755 airflow/api_connexion/schemas/version_schema.py mode change 100644 => 100755 airflow/api_connexion/schemas/xcom_schema.py mode change 100644 => 100755 airflow/api_connexion/security.py mode change 100644 => 100755 airflow/cli/__init__.py mode change 100644 => 100755 airflow/cli/cli_parser.py mode change 100644 => 100755 airflow/cli/commands/__init__.py mode change 100644 => 100755 airflow/cli/commands/celery_command.py mode change 100644 => 100755 airflow/cli/commands/config_command.py mode change 100644 => 100755 airflow/cli/commands/connection_command.py mode change 100644 => 100755 airflow/cli/commands/dag_command.py mode change 100644 => 100755 airflow/cli/commands/db_command.py mode change 100644 => 100755 airflow/cli/commands/info_command.py mode change 100644 => 100755 airflow/cli/commands/kerberos_command.py mode change 100644 => 100755 airflow/cli/commands/legacy_commands.py mode change 100644 => 100755 airflow/cli/commands/plugins_command.py mode change 100644 => 100755 airflow/cli/commands/pool_command.py mode change 100644 => 100755 airflow/cli/commands/role_command.py mode change 100644 => 100755 airflow/cli/commands/rotate_fernet_key_command.py mode change 100644 => 100755 airflow/cli/commands/scheduler_command.py mode change 100644 => 100755 airflow/cli/commands/sync_perm_command.py mode change 100644 => 100755 airflow/cli/commands/task_command.py mode change 100644 => 100755 airflow/cli/commands/user_command.py mode change 100644 => 100755 airflow/cli/commands/variable_command.py mode change 100644 => 100755 airflow/cli/commands/version_command.py mode change 100644 => 100755 airflow/cli/commands/webserver_command.py mode change 100644 => 100755 airflow/config_templates/__init__.py mode change 100644 => 100755 airflow/config_templates/airflow_local_settings.py mode change 100644 => 100755 airflow/config_templates/config.yml mode change 100644 => 100755 airflow/config_templates/default_airflow.cfg mode change 100644 => 100755 airflow/config_templates/default_celery.py mode change 100644 => 100755 airflow/config_templates/default_test.cfg mode change 100644 => 100755 airflow/config_templates/default_webserver_config.py mode change 100644 => 100755 airflow/configuration.py mode change 100644 => 100755 airflow/contrib/__init__.py mode change 100644 => 100755 airflow/contrib/hooks/__init__.py mode change 100644 => 100755 airflow/contrib/hooks/aws_athena_hook.py mode change 100644 => 100755 airflow/contrib/hooks/aws_datasync_hook.py mode change 100644 => 100755 airflow/contrib/hooks/aws_dynamodb_hook.py mode change 100644 => 100755 airflow/contrib/hooks/aws_firehose_hook.py mode change 100644 => 100755 airflow/contrib/hooks/aws_glue_catalog_hook.py mode change 100644 => 100755 airflow/contrib/hooks/aws_hook.py mode change 100644 => 100755 airflow/contrib/hooks/aws_lambda_hook.py mode change 100644 => 100755 airflow/contrib/hooks/aws_logs_hook.py mode change 100644 => 100755 airflow/contrib/hooks/aws_sns_hook.py mode change 100644 => 100755 airflow/contrib/hooks/aws_sqs_hook.py mode change 100644 => 100755 airflow/contrib/hooks/azure_container_instance_hook.py mode change 100644 => 100755 airflow/contrib/hooks/azure_container_registry_hook.py mode change 100644 => 100755 airflow/contrib/hooks/azure_container_volume_hook.py mode change 100644 => 100755 airflow/contrib/hooks/azure_cosmos_hook.py mode change 100644 => 100755 airflow/contrib/hooks/azure_data_lake_hook.py mode change 100644 => 100755 airflow/contrib/hooks/azure_fileshare_hook.py mode change 100644 => 100755 airflow/contrib/hooks/bigquery_hook.py mode change 100644 => 100755 airflow/contrib/hooks/cassandra_hook.py mode change 100644 => 100755 airflow/contrib/hooks/cloudant_hook.py mode change 100644 => 100755 airflow/contrib/hooks/databricks_hook.py mode change 100644 => 100755 airflow/contrib/hooks/datadog_hook.py mode change 100644 => 100755 airflow/contrib/hooks/datastore_hook.py mode change 100644 => 100755 airflow/contrib/hooks/dingding_hook.py mode change 100644 => 100755 airflow/contrib/hooks/discord_webhook_hook.py mode change 100644 => 100755 airflow/contrib/hooks/emr_hook.py mode change 100644 => 100755 airflow/contrib/hooks/fs_hook.py mode change 100644 => 100755 airflow/contrib/hooks/ftp_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_api_base_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_bigtable_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_cloud_build_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_compute_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_container_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_dataflow_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_dataproc_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_dlp_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_function_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_kms_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_mlengine_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_natural_language_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_pubsub_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_spanner_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_speech_to_text_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_sql_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_tasks_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_text_to_speech_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_transfer_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_translate_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_video_intelligence_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcp_vision_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gcs_hook.py mode change 100644 => 100755 airflow/contrib/hooks/gdrive_hook.py mode change 100644 => 100755 airflow/contrib/hooks/grpc_hook.py mode change 100644 => 100755 airflow/contrib/hooks/imap_hook.py mode change 100644 => 100755 airflow/contrib/hooks/jenkins_hook.py mode change 100644 => 100755 airflow/contrib/hooks/jira_hook.py mode change 100644 => 100755 airflow/contrib/hooks/mongo_hook.py mode change 100644 => 100755 airflow/contrib/hooks/openfaas_hook.py mode change 100644 => 100755 airflow/contrib/hooks/opsgenie_alert_hook.py mode change 100644 => 100755 airflow/contrib/hooks/pagerduty_hook.py mode change 100644 => 100755 airflow/contrib/hooks/pinot_hook.py mode change 100644 => 100755 airflow/contrib/hooks/qubole_check_hook.py mode change 100644 => 100755 airflow/contrib/hooks/qubole_hook.py mode change 100644 => 100755 airflow/contrib/hooks/redis_hook.py mode change 100644 => 100755 airflow/contrib/hooks/redshift_hook.py mode change 100644 => 100755 airflow/contrib/hooks/sagemaker_hook.py mode change 100644 => 100755 airflow/contrib/hooks/salesforce_hook.py mode change 100644 => 100755 airflow/contrib/hooks/segment_hook.py mode change 100644 => 100755 airflow/contrib/hooks/sftp_hook.py mode change 100644 => 100755 airflow/contrib/hooks/slack_webhook_hook.py mode change 100644 => 100755 airflow/contrib/hooks/snowflake_hook.py mode change 100644 => 100755 airflow/contrib/hooks/spark_jdbc_hook.py mode change 100644 => 100755 airflow/contrib/hooks/spark_sql_hook.py mode change 100644 => 100755 airflow/contrib/hooks/spark_submit_hook.py mode change 100644 => 100755 airflow/contrib/hooks/sqoop_hook.py mode change 100644 => 100755 airflow/contrib/hooks/ssh_hook.py mode change 100644 => 100755 airflow/contrib/hooks/vertica_hook.py mode change 100644 => 100755 airflow/contrib/hooks/wasb_hook.py mode change 100644 => 100755 airflow/contrib/hooks/winrm_hook.py mode change 100644 => 100755 airflow/contrib/operators/__init__.py mode change 100644 => 100755 airflow/contrib/operators/adls_list_operator.py mode change 100644 => 100755 airflow/contrib/operators/adls_to_gcs.py mode change 100644 => 100755 airflow/contrib/operators/aws_athena_operator.py mode change 100644 => 100755 airflow/contrib/operators/aws_sqs_publish_operator.py mode change 100644 => 100755 airflow/contrib/operators/awsbatch_operator.py mode change 100644 => 100755 airflow/contrib/operators/azure_container_instances_operator.py mode change 100644 => 100755 airflow/contrib/operators/azure_cosmos_operator.py mode change 100644 => 100755 airflow/contrib/operators/bigquery_check_operator.py mode change 100644 => 100755 airflow/contrib/operators/bigquery_get_data.py mode change 100644 => 100755 airflow/contrib/operators/bigquery_operator.py mode change 100644 => 100755 airflow/contrib/operators/bigquery_table_delete_operator.py mode change 100644 => 100755 airflow/contrib/operators/bigquery_to_bigquery.py mode change 100644 => 100755 airflow/contrib/operators/bigquery_to_gcs.py mode change 100644 => 100755 airflow/contrib/operators/bigquery_to_mysql_operator.py mode change 100644 => 100755 airflow/contrib/operators/cassandra_to_gcs.py mode change 100644 => 100755 airflow/contrib/operators/databricks_operator.py mode change 100644 => 100755 airflow/contrib/operators/dataflow_operator.py mode change 100644 => 100755 airflow/contrib/operators/dataproc_operator.py mode change 100644 => 100755 airflow/contrib/operators/datastore_export_operator.py mode change 100644 => 100755 airflow/contrib/operators/datastore_import_operator.py mode change 100644 => 100755 airflow/contrib/operators/dingding_operator.py mode change 100644 => 100755 airflow/contrib/operators/discord_webhook_operator.py mode change 100644 => 100755 airflow/contrib/operators/docker_swarm_operator.py mode change 100644 => 100755 airflow/contrib/operators/druid_operator.py mode change 100644 => 100755 airflow/contrib/operators/dynamodb_to_s3.py mode change 100644 => 100755 airflow/contrib/operators/ecs_operator.py mode change 100644 => 100755 airflow/contrib/operators/emr_add_steps_operator.py mode change 100644 => 100755 airflow/contrib/operators/emr_create_job_flow_operator.py mode change 100644 => 100755 airflow/contrib/operators/emr_terminate_job_flow_operator.py mode change 100644 => 100755 airflow/contrib/operators/file_to_gcs.py mode change 100644 => 100755 airflow/contrib/operators/file_to_wasb.py mode change 100644 => 100755 airflow/contrib/operators/gcp_bigtable_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_cloud_build_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_compute_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_container_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_dlp_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_function_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_natural_language_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_spanner_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_speech_to_text_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_sql_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_tasks_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_text_to_speech_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_transfer_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_translate_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_translate_speech_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_video_intelligence_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcp_vision_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcs_acl_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcs_delete_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcs_download_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcs_list_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcs_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcs_to_bq.py mode change 100644 => 100755 airflow/contrib/operators/gcs_to_gcs.py mode change 100644 => 100755 airflow/contrib/operators/gcs_to_gcs_transfer_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcs_to_gdrive_operator.py mode change 100644 => 100755 airflow/contrib/operators/gcs_to_s3.py mode change 100644 => 100755 airflow/contrib/operators/grpc_operator.py mode change 100644 => 100755 airflow/contrib/operators/hive_to_dynamodb.py mode change 100644 => 100755 airflow/contrib/operators/imap_attachment_to_s3_operator.py mode change 100644 => 100755 airflow/contrib/operators/jenkins_job_trigger_operator.py mode change 100644 => 100755 airflow/contrib/operators/jira_operator.py mode change 100644 => 100755 airflow/contrib/operators/kubernetes_pod_operator.py mode change 100644 => 100755 airflow/contrib/operators/mlengine_operator.py mode change 100644 => 100755 airflow/contrib/operators/mongo_to_s3.py mode change 100644 => 100755 airflow/contrib/operators/mssql_to_gcs.py mode change 100644 => 100755 airflow/contrib/operators/mysql_to_gcs.py mode change 100644 => 100755 airflow/contrib/operators/opsgenie_alert_operator.py mode change 100644 => 100755 airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py mode change 100644 => 100755 airflow/contrib/operators/oracle_to_oracle_transfer.py mode change 100644 => 100755 airflow/contrib/operators/postgres_to_gcs_operator.py mode change 100644 => 100755 airflow/contrib/operators/pubsub_operator.py mode change 100644 => 100755 airflow/contrib/operators/qubole_check_operator.py mode change 100644 => 100755 airflow/contrib/operators/qubole_operator.py mode change 100644 => 100755 airflow/contrib/operators/redis_publish_operator.py mode change 100644 => 100755 airflow/contrib/operators/s3_copy_object_operator.py mode change 100644 => 100755 airflow/contrib/operators/s3_delete_objects_operator.py mode change 100644 => 100755 airflow/contrib/operators/s3_list_operator.py mode change 100644 => 100755 airflow/contrib/operators/s3_to_gcs_operator.py mode change 100644 => 100755 airflow/contrib/operators/s3_to_gcs_transfer_operator.py mode change 100644 => 100755 airflow/contrib/operators/s3_to_sftp_operator.py mode change 100644 => 100755 airflow/contrib/operators/sagemaker_base_operator.py mode change 100644 => 100755 airflow/contrib/operators/sagemaker_endpoint_config_operator.py mode change 100644 => 100755 airflow/contrib/operators/sagemaker_endpoint_operator.py mode change 100644 => 100755 airflow/contrib/operators/sagemaker_model_operator.py mode change 100644 => 100755 airflow/contrib/operators/sagemaker_training_operator.py mode change 100644 => 100755 airflow/contrib/operators/sagemaker_transform_operator.py mode change 100644 => 100755 airflow/contrib/operators/sagemaker_tuning_operator.py mode change 100644 => 100755 airflow/contrib/operators/segment_track_event_operator.py mode change 100644 => 100755 airflow/contrib/operators/sftp_operator.py mode change 100644 => 100755 airflow/contrib/operators/sftp_to_s3_operator.py mode change 100644 => 100755 airflow/contrib/operators/slack_webhook_operator.py mode change 100644 => 100755 airflow/contrib/operators/snowflake_operator.py mode change 100644 => 100755 airflow/contrib/operators/sns_publish_operator.py mode change 100644 => 100755 airflow/contrib/operators/spark_jdbc_operator.py mode change 100644 => 100755 airflow/contrib/operators/spark_sql_operator.py mode change 100644 => 100755 airflow/contrib/operators/spark_submit_operator.py mode change 100644 => 100755 airflow/contrib/operators/sql_to_gcs.py mode change 100644 => 100755 airflow/contrib/operators/sqoop_operator.py mode change 100644 => 100755 airflow/contrib/operators/ssh_operator.py mode change 100644 => 100755 airflow/contrib/operators/vertica_operator.py mode change 100644 => 100755 airflow/contrib/operators/vertica_to_hive.py mode change 100644 => 100755 airflow/contrib/operators/vertica_to_mysql.py mode change 100644 => 100755 airflow/contrib/operators/wasb_delete_blob_operator.py mode change 100644 => 100755 airflow/contrib/operators/winrm_operator.py mode change 100644 => 100755 airflow/contrib/secrets/__init__.py mode change 100644 => 100755 airflow/contrib/secrets/aws_secrets_manager.py mode change 100644 => 100755 airflow/contrib/secrets/aws_systems_manager.py mode change 100644 => 100755 airflow/contrib/secrets/gcp_secrets_manager.py mode change 100644 => 100755 airflow/contrib/secrets/hashicorp_vault.py mode change 100644 => 100755 airflow/contrib/sensors/__init__.py mode change 100644 => 100755 airflow/contrib/sensors/aws_athena_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/aws_glue_catalog_partition_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/aws_redshift_cluster_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/aws_sqs_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/azure_cosmos_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/bash_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/bigquery_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/cassandra_record_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/cassandra_table_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/celery_queue_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/datadog_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/emr_base_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/emr_job_flow_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/emr_step_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/file_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/ftp_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/gcp_transfer_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/gcs_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/hdfs_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/imap_attachment_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/jira_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/mongo_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/pubsub_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/python_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/qubole_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/redis_key_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/redis_pub_sub_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/sagemaker_base_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/sagemaker_endpoint_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/sagemaker_training_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/sagemaker_transform_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/sagemaker_tuning_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/sftp_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/wasb_sensor.py mode change 100644 => 100755 airflow/contrib/sensors/weekday_sensor.py mode change 100644 => 100755 airflow/contrib/task_runner/__init__.py mode change 100644 => 100755 airflow/contrib/task_runner/cgroup_task_runner.py mode change 100644 => 100755 airflow/contrib/utils/__init__.py mode change 100644 => 100755 airflow/contrib/utils/gcp_field_sanitizer.py mode change 100644 => 100755 airflow/contrib/utils/gcp_field_validator.py mode change 100644 => 100755 airflow/contrib/utils/log/__init__.py mode change 100644 => 100755 airflow/contrib/utils/log/task_handler_with_custom_formatter.py mode change 100644 => 100755 airflow/contrib/utils/mlengine_operator_utils.py mode change 100644 => 100755 airflow/contrib/utils/mlengine_prediction_summary.py mode change 100644 => 100755 airflow/contrib/utils/sendgrid.py mode change 100644 => 100755 airflow/contrib/utils/weekday.py mode change 100644 => 100755 airflow/dag/__init__.py mode change 100644 => 100755 airflow/dag/base_dag.py mode change 100644 => 100755 airflow/decorators.py mode change 100644 => 100755 airflow/example_dags/__init__.py mode change 100644 => 100755 airflow/example_dags/example_bash_operator.py mode change 100644 => 100755 airflow/example_dags/example_branch_operator.py mode change 100644 => 100755 airflow/example_dags/example_branch_python_dop_operator_3.py mode change 100644 => 100755 airflow/example_dags/example_complex.py mode change 100644 => 100755 airflow/example_dags/example_external_task_marker_dag.py mode change 100644 => 100755 airflow/example_dags/example_kubernetes_executor.py mode change 100644 => 100755 airflow/example_dags/example_kubernetes_executor_config.py mode change 100644 => 100755 airflow/example_dags/example_latest_only.py mode change 100644 => 100755 airflow/example_dags/example_latest_only_with_trigger.py mode change 100644 => 100755 airflow/example_dags/example_nested_branch_dag.py mode change 100644 => 100755 airflow/example_dags/example_passing_params_via_test_command.py mode change 100644 => 100755 airflow/example_dags/example_python_operator.py mode change 100644 => 100755 airflow/example_dags/example_short_circuit_operator.py mode change 100644 => 100755 airflow/example_dags/example_skip_dag.py mode change 100644 => 100755 airflow/example_dags/example_subdag_operator.py mode change 100644 => 100755 airflow/example_dags/example_trigger_controller_dag.py mode change 100644 => 100755 airflow/example_dags/example_trigger_target_dag.py mode change 100644 => 100755 airflow/example_dags/example_xcom.py mode change 100644 => 100755 airflow/example_dags/example_xcomargs.py mode change 100644 => 100755 airflow/example_dags/input_notebook.ipynb mode change 100644 => 100755 airflow/example_dags/libs/__init__.py mode change 100644 => 100755 airflow/example_dags/libs/helper.py mode change 100644 => 100755 airflow/example_dags/subdags/__init__.py mode change 100644 => 100755 airflow/example_dags/subdags/subdag.py mode change 100644 => 100755 airflow/example_dags/test_utils.py mode change 100644 => 100755 airflow/example_dags/tutorial.py mode change 100644 => 100755 airflow/exceptions.py mode change 100644 => 100755 airflow/executors/__init__.py mode change 100644 => 100755 airflow/executors/base_executor.py mode change 100644 => 100755 airflow/executors/celery_executor.py mode change 100644 => 100755 airflow/executors/dask_executor.py mode change 100644 => 100755 airflow/executors/debug_executor.py mode change 100644 => 100755 airflow/executors/executor_loader.py mode change 100644 => 100755 airflow/executors/kubernetes_executor.py mode change 100644 => 100755 airflow/executors/local_executor.py mode change 100644 => 100755 airflow/executors/sequential_executor.py mode change 100644 => 100755 airflow/hooks/S3_hook.py mode change 100644 => 100755 airflow/hooks/__init__.py mode change 100644 => 100755 airflow/hooks/base_hook.py mode change 100644 => 100755 airflow/hooks/dbapi_hook.py mode change 100644 => 100755 airflow/hooks/docker_hook.py mode change 100644 => 100755 airflow/hooks/druid_hook.py mode change 100644 => 100755 airflow/hooks/filesystem.py mode change 100644 => 100755 airflow/hooks/hdfs_hook.py mode change 100644 => 100755 airflow/hooks/hive_hooks.py mode change 100644 => 100755 airflow/hooks/http_hook.py mode change 100644 => 100755 airflow/hooks/jdbc_hook.py mode change 100644 => 100755 airflow/hooks/mssql_hook.py mode change 100644 => 100755 airflow/hooks/mysql_hook.py mode change 100644 => 100755 airflow/hooks/oracle_hook.py mode change 100644 => 100755 airflow/hooks/pig_hook.py mode change 100644 => 100755 airflow/hooks/postgres_hook.py mode change 100644 => 100755 airflow/hooks/presto_hook.py mode change 100644 => 100755 airflow/hooks/samba_hook.py mode change 100644 => 100755 airflow/hooks/slack_hook.py mode change 100644 => 100755 airflow/hooks/sqlite_hook.py mode change 100644 => 100755 airflow/hooks/webhdfs_hook.py mode change 100644 => 100755 airflow/hooks/zendesk_hook.py mode change 100644 => 100755 airflow/jobs/__init__.py mode change 100644 => 100755 airflow/jobs/backfill_job.py mode change 100644 => 100755 airflow/jobs/base_job.py mode change 100644 => 100755 airflow/jobs/local_task_job.py mode change 100644 => 100755 airflow/jobs/scheduler_job.py mode change 100644 => 100755 airflow/kubernetes/__init__.py mode change 100644 => 100755 airflow/kubernetes/k8s_model.py mode change 100644 => 100755 airflow/kubernetes/kube_client.py mode change 100644 => 100755 airflow/kubernetes/pod.py mode change 100644 => 100755 airflow/kubernetes/pod_generator.py mode change 100644 => 100755 airflow/kubernetes/pod_launcher.py mode change 100644 => 100755 airflow/kubernetes/pod_runtime_info_env.py mode change 100644 => 100755 airflow/kubernetes/refresh_config.py mode change 100644 => 100755 airflow/kubernetes/secret.py mode change 100644 => 100755 airflow/kubernetes/volume.py mode change 100644 => 100755 airflow/kubernetes/volume_mount.py mode change 100644 => 100755 airflow/kubernetes/worker_configuration.py mode change 100644 => 100755 airflow/lineage/__init__.py mode change 100644 => 100755 airflow/lineage/entities.py mode change 100644 => 100755 airflow/logging_config.py mode change 100644 => 100755 airflow/macros/__init__.py mode change 100644 => 100755 airflow/macros/hive.py mode change 100644 => 100755 airflow/migrations/__init__.py mode change 100644 => 100755 airflow/migrations/env.py mode change 100644 => 100755 airflow/migrations/script.py.mako mode change 100644 => 100755 airflow/migrations/versions/004c1210f153_increase_queue_name_size_limit.py mode change 100644 => 100755 airflow/migrations/versions/03bc53e68815_add_sm_dag_index.py mode change 100644 => 100755 airflow/migrations/versions/05f30312d566_merge_heads.py mode change 100644 => 100755 airflow/migrations/versions/0a2a5b66e19d_add_task_reschedule_table.py mode change 100644 => 100755 airflow/migrations/versions/0e2a74e0fc9f_add_time_zone_awareness.py mode change 100644 => 100755 airflow/migrations/versions/127d2bf2dfa7_add_dag_id_state_index_on_dag_run_table.py mode change 100644 => 100755 airflow/migrations/versions/13eb55f81627_for_compatibility.py mode change 100644 => 100755 airflow/migrations/versions/1507a7289a2f_create_is_encrypted.py mode change 100644 => 100755 airflow/migrations/versions/1968acfc09e3_add_is_encrypted_column_to_variable_.py mode change 100644 => 100755 airflow/migrations/versions/1b38cef5b76e_add_dagrun.py mode change 100644 => 100755 airflow/migrations/versions/211e584da130_add_ti_state_index.py mode change 100644 => 100755 airflow/migrations/versions/27c6a30d7c24_add_executor_config_to_task_instance.py mode change 100644 => 100755 airflow/migrations/versions/2e541a1dcfed_task_duration.py mode change 100644 => 100755 airflow/migrations/versions/2e82aab8ef20_rename_user_table.py mode change 100644 => 100755 airflow/migrations/versions/338e90f54d61_more_logging_into_task_isntance.py mode change 100644 => 100755 airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py mode change 100644 => 100755 airflow/migrations/versions/3c20cacc0044_add_dagrun_run_type.py mode change 100644 => 100755 airflow/migrations/versions/40e67319e3a9_dagrun_config.py mode change 100644 => 100755 airflow/migrations/versions/41f5f12752f8_add_superuser_field.py mode change 100644 => 100755 airflow/migrations/versions/4446e08588_dagrun_start_end.py mode change 100644 => 100755 airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py mode change 100644 => 100755 airflow/migrations/versions/502898887f84_adding_extra_to_log.py mode change 100644 => 100755 airflow/migrations/versions/52d714495f0_job_id_indices.py mode change 100644 => 100755 airflow/migrations/versions/561833c1c74b_add_password_column_to_user.py mode change 100644 => 100755 airflow/migrations/versions/5e7d17757c7a_add_pid_field_to_taskinstance.py mode change 100644 => 100755 airflow/migrations/versions/64de9cddf6c9_add_task_fails_journal_table.py mode change 100644 => 100755 airflow/migrations/versions/6e96a59344a4_make_taskinstance_pool_not_nullable.py mode change 100644 => 100755 airflow/migrations/versions/74effc47d867_change_datetime_to_datetime2_6_on_mssql_.py mode change 100644 => 100755 airflow/migrations/versions/7939bcff74ba_add_dagtags_table.py mode change 100644 => 100755 airflow/migrations/versions/8504051e801b_xcom_dag_task_indices.py mode change 100644 => 100755 airflow/migrations/versions/852ae6c715af_add_rendered_task_instance_fields_table.py mode change 100644 => 100755 airflow/migrations/versions/856955da8476_fix_sqlite_foreign_key.py mode change 100644 => 100755 airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py mode change 100644 => 100755 airflow/migrations/versions/8d48763f6d53_add_unique_constraint_to_conn_id.py mode change 100644 => 100755 airflow/migrations/versions/8f966b9c467a_set_conn_type_as_non_nullable.py mode change 100644 => 100755 airflow/migrations/versions/939bb1e647c8_task_reschedule_fk_on_cascade_delete.py mode change 100644 => 100755 airflow/migrations/versions/947454bf1dff_add_ti_job_id_index.py mode change 100644 => 100755 airflow/migrations/versions/952da73b5eff_add_dag_code_table.py mode change 100644 => 100755 airflow/migrations/versions/9635ae0956e7_index_faskfail.py mode change 100644 => 100755 airflow/migrations/versions/__init__.py mode change 100644 => 100755 airflow/migrations/versions/a4c2fd67d16b_add_pool_slots_field_to_task_instance.py mode change 100644 => 100755 airflow/migrations/versions/a56c9515abdc_remove_dag_stat_table.py mode change 100644 => 100755 airflow/migrations/versions/a66efa278eea_add_precision_to_execution_date_in_mysql.py mode change 100644 => 100755 airflow/migrations/versions/b0125267960b_merge_heads.py mode change 100644 => 100755 airflow/migrations/versions/b25a55525161_increase_length_of_pool_name.py mode change 100644 => 100755 airflow/migrations/versions/b3b105409875_add_root_dag_id_to_dag.py mode change 100644 => 100755 airflow/migrations/versions/bba5a7cfc896_add_a_column_to_track_the_encryption_.py mode change 100644 => 100755 airflow/migrations/versions/bbc73705a13e_add_notification_sent_column_to_sla_miss.py mode change 100644 => 100755 airflow/migrations/versions/bbf4a7ad0465_remove_id_column_from_xcom.py mode change 100644 => 100755 airflow/migrations/versions/bdaa763e6c56_make_xcom_value_column_a_large_binary.py mode change 100644 => 100755 airflow/migrations/versions/bf00311e1990_add_index_to_taskinstance.py mode change 100644 => 100755 airflow/migrations/versions/c8ffec048a3b_add_fields_to_dag.py mode change 100644 => 100755 airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py mode change 100644 => 100755 airflow/migrations/versions/cf5dc11e79ad_drop_user_and_chart.py mode change 100644 => 100755 airflow/migrations/versions/d2ae31099d61_increase_text_size_for_mysql.py mode change 100644 => 100755 airflow/migrations/versions/d38e04c12aa2_add_serialized_dag_table.py mode change 100644 => 100755 airflow/migrations/versions/da3f683c3a5a_add_dag_hash_column_to_serialized_dag_.py mode change 100644 => 100755 airflow/migrations/versions/dd25f486b8ea_add_idx_log_dag.py mode change 100644 => 100755 airflow/migrations/versions/dd4ecb8fbee3_add_schedule_interval_to_dag.py mode change 100644 => 100755 airflow/migrations/versions/e3a246e0dc1_current_schema.py mode change 100644 => 100755 airflow/migrations/versions/f23433877c24_fix_mysql_not_null_constraint.py mode change 100644 => 100755 airflow/migrations/versions/f2ca10b85618_add_dag_stats_table.py mode change 100644 => 100755 airflow/migrations/versions/fe461863935f_increase_length_for_connection_password.py mode change 100644 => 100755 airflow/models/__init__.py mode change 100644 => 100755 airflow/models/base.py mode change 100644 => 100755 airflow/models/baseoperator.py mode change 100644 => 100755 airflow/models/connection.py mode change 100644 => 100755 airflow/models/crypto.py mode change 100644 => 100755 airflow/models/dag.py mode change 100644 => 100755 airflow/models/dagbag.py mode change 100644 => 100755 airflow/models/dagcode.py mode change 100644 => 100755 airflow/models/dagpickle.py mode change 100644 => 100755 airflow/models/dagrun.py mode change 100644 => 100755 airflow/models/errors.py mode change 100644 => 100755 airflow/models/kubernetes.py mode change 100644 => 100755 airflow/models/log.py mode change 100644 => 100755 airflow/models/pool.py mode change 100644 => 100755 airflow/models/renderedtifields.py mode change 100644 => 100755 airflow/models/serialized_dag.py mode change 100644 => 100755 airflow/models/skipmixin.py mode change 100644 => 100755 airflow/models/slamiss.py mode change 100644 => 100755 airflow/models/taskfail.py mode change 100644 => 100755 airflow/models/taskinstance.py mode change 100644 => 100755 airflow/models/taskreschedule.py mode change 100644 => 100755 airflow/models/variable.py mode change 100644 => 100755 airflow/models/xcom.py mode change 100644 => 100755 airflow/models/xcom_arg.py mode change 100644 => 100755 airflow/mypy/__init__.py mode change 100644 => 100755 airflow/mypy/plugin/__init__.py mode change 100644 => 100755 airflow/mypy/plugin/decorators.py mode change 100644 => 100755 airflow/operators/__init__.py mode change 100644 => 100755 airflow/operators/bash.py mode change 100644 => 100755 airflow/operators/bash_operator.py mode change 100644 => 100755 airflow/operators/branch_operator.py mode change 100644 => 100755 airflow/operators/check_operator.py mode change 100644 => 100755 airflow/operators/dagrun_operator.py mode change 100644 => 100755 airflow/operators/docker_operator.py mode change 100644 => 100755 airflow/operators/druid_check_operator.py mode change 100644 => 100755 airflow/operators/dummy_operator.py mode change 100644 => 100755 airflow/operators/email.py mode change 100644 => 100755 airflow/operators/email_operator.py mode change 100644 => 100755 airflow/operators/gcs_to_s3.py mode change 100644 => 100755 airflow/operators/generic_transfer.py mode change 100644 => 100755 airflow/operators/google_api_to_s3_transfer.py mode change 100644 => 100755 airflow/operators/hive_operator.py mode change 100644 => 100755 airflow/operators/hive_stats_operator.py mode change 100644 => 100755 airflow/operators/hive_to_druid.py mode change 100644 => 100755 airflow/operators/hive_to_mysql.py mode change 100644 => 100755 airflow/operators/hive_to_samba_operator.py mode change 100644 => 100755 airflow/operators/http_operator.py mode change 100644 => 100755 airflow/operators/jdbc_operator.py mode change 100644 => 100755 airflow/operators/latest_only_operator.py mode change 100644 => 100755 airflow/operators/mssql_operator.py mode change 100644 => 100755 airflow/operators/mssql_to_hive.py mode change 100644 => 100755 airflow/operators/mysql_operator.py mode change 100644 => 100755 airflow/operators/mysql_to_hive.py mode change 100644 => 100755 airflow/operators/oracle_operator.py mode change 100644 => 100755 airflow/operators/papermill_operator.py mode change 100644 => 100755 airflow/operators/pig_operator.py mode change 100644 => 100755 airflow/operators/postgres_operator.py mode change 100644 => 100755 airflow/operators/presto_check_operator.py mode change 100644 => 100755 airflow/operators/presto_to_mysql.py mode change 100644 => 100755 airflow/operators/python.py mode change 100644 => 100755 airflow/operators/python_operator.py mode change 100644 => 100755 airflow/operators/redshift_to_s3_operator.py mode change 100644 => 100755 airflow/operators/s3_file_transform_operator.py mode change 100644 => 100755 airflow/operators/s3_to_hive_operator.py mode change 100644 => 100755 airflow/operators/s3_to_redshift_operator.py mode change 100644 => 100755 airflow/operators/slack_operator.py mode change 100644 => 100755 airflow/operators/sql.py mode change 100644 => 100755 airflow/operators/sql_branch_operator.py mode change 100644 => 100755 airflow/operators/sqlite_operator.py mode change 100644 => 100755 airflow/operators/subdag_operator.py mode change 100644 => 100755 airflow/plugins_manager.py mode change 100644 => 100755 airflow/providers/__init__.py mode change 100644 => 100755 airflow/providers/amazon/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/amazon/README.md mode change 100644 => 100755 airflow/providers/amazon/__init__.py mode change 100644 => 100755 airflow/providers/amazon/aws/__init__.py mode change 100644 => 100755 airflow/providers/amazon/aws/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/amazon/aws/example_dags/example_datasync_1.py mode change 100644 => 100755 airflow/providers/amazon/aws/example_dags/example_datasync_2.py mode change 100644 => 100755 airflow/providers/amazon/aws/example_dags/example_ecs_fargate.py mode change 100644 => 100755 airflow/providers/amazon/aws/example_dags/example_emr_job_flow_automatic_steps.py mode change 100644 => 100755 airflow/providers/amazon/aws/example_dags/example_emr_job_flow_manual_steps.py mode change 100644 => 100755 airflow/providers/amazon/aws/example_dags/example_google_api_to_s3_transfer_advanced.py mode change 100644 => 100755 airflow/providers/amazon/aws/example_dags/example_google_api_to_s3_transfer_basic.py mode change 100644 => 100755 airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py mode change 100644 => 100755 airflow/providers/amazon/aws/example_dags/example_s3_bucket.py mode change 100644 => 100755 airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/__init__.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/athena.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/aws_dynamodb.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/base_aws.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/batch_client.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/batch_waiters.json mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/batch_waiters.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/cloud_formation.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/datasync.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/ec2.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/emr.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/glue.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/glue_catalog.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/kinesis.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/lambda_function.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/logs.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/redshift.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/s3.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/sagemaker.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/ses.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/sns.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/sqs.py mode change 100644 => 100755 airflow/providers/amazon/aws/hooks/step_function.py mode change 100644 => 100755 airflow/providers/amazon/aws/log/__init__.py mode change 100644 => 100755 airflow/providers/amazon/aws/log/cloudwatch_task_handler.py mode change 100644 => 100755 airflow/providers/amazon/aws/log/s3_task_handler.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/__init__.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/athena.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/batch.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/cloud_formation.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/datasync.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/ec2_start_instance.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/ec2_stop_instance.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/ecs.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/emr_add_steps.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/emr_create_job_flow.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/emr_modify_cluster.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/emr_terminate_job_flow.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/glue.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/s3_bucket.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/s3_copy_object.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/s3_delete_objects.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/s3_file_transform.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/s3_list.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/sagemaker_base.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/sagemaker_endpoint.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/sagemaker_endpoint_config.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/sagemaker_model.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/sagemaker_processing.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/sagemaker_training.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/sagemaker_transform.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/sagemaker_tuning.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/sns.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/sqs.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/step_function_get_execution_output.py mode change 100644 => 100755 airflow/providers/amazon/aws/operators/step_function_start_execution.py mode change 100644 => 100755 airflow/providers/amazon/aws/secrets/__init__.py mode change 100644 => 100755 airflow/providers/amazon/aws/secrets/secrets_manager.py mode change 100644 => 100755 airflow/providers/amazon/aws/secrets/systems_manager.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/__init__.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/athena.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/cloud_formation.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/ec2_instance_state.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/emr_base.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/emr_job_flow.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/emr_step.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/glue.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/glue_catalog_partition.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/redshift.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/s3_key.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/s3_keys_unchanged.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/s3_prefix.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/sagemaker_base.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/sagemaker_endpoint.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/sagemaker_training.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/sagemaker_transform.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/sagemaker_tuning.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/sqs.py mode change 100644 => 100755 airflow/providers/amazon/aws/sensors/step_function_execution.py mode change 100644 => 100755 airflow/providers/amazon/aws/transfers/__init__.py mode change 100644 => 100755 airflow/providers/amazon/aws/transfers/dynamodb_to_s3.py mode change 100644 => 100755 airflow/providers/amazon/aws/transfers/gcs_to_s3.py mode change 100644 => 100755 airflow/providers/amazon/aws/transfers/google_api_to_s3.py mode change 100644 => 100755 airflow/providers/amazon/aws/transfers/hive_to_dynamodb.py mode change 100644 => 100755 airflow/providers/amazon/aws/transfers/imap_attachment_to_s3.py mode change 100644 => 100755 airflow/providers/amazon/aws/transfers/mongo_to_s3.py mode change 100644 => 100755 airflow/providers/amazon/aws/transfers/mysql_to_s3.py mode change 100644 => 100755 airflow/providers/amazon/aws/transfers/redshift_to_s3.py mode change 100644 => 100755 airflow/providers/amazon/aws/transfers/s3_to_redshift.py mode change 100644 => 100755 airflow/providers/amazon/aws/transfers/s3_to_sftp.py mode change 100644 => 100755 airflow/providers/amazon/aws/transfers/sftp_to_s3.py mode change 100644 => 100755 airflow/providers/apache/__init__.py mode change 100644 => 100755 airflow/providers/apache/cassandra/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/apache/cassandra/README.md mode change 100644 => 100755 airflow/providers/apache/cassandra/__init__.py mode change 100644 => 100755 airflow/providers/apache/cassandra/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/apache/cassandra/example_dags/example_cassandra_dag.py mode change 100644 => 100755 airflow/providers/apache/cassandra/hooks/__init__.py mode change 100644 => 100755 airflow/providers/apache/cassandra/hooks/cassandra.py mode change 100644 => 100755 airflow/providers/apache/cassandra/sensors/__init__.py mode change 100644 => 100755 airflow/providers/apache/cassandra/sensors/record.py mode change 100644 => 100755 airflow/providers/apache/cassandra/sensors/table.py mode change 100644 => 100755 airflow/providers/apache/druid/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/apache/druid/README.md mode change 100644 => 100755 airflow/providers/apache/druid/__init__.py mode change 100644 => 100755 airflow/providers/apache/druid/hooks/__init__.py mode change 100644 => 100755 airflow/providers/apache/druid/hooks/druid.py mode change 100644 => 100755 airflow/providers/apache/druid/operators/__init__.py mode change 100644 => 100755 airflow/providers/apache/druid/operators/druid.py mode change 100644 => 100755 airflow/providers/apache/druid/operators/druid_check.py mode change 100644 => 100755 airflow/providers/apache/druid/transfers/__init__.py mode change 100644 => 100755 airflow/providers/apache/druid/transfers/hive_to_druid.py mode change 100644 => 100755 airflow/providers/apache/hdfs/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/apache/hdfs/README.md mode change 100644 => 100755 airflow/providers/apache/hdfs/__init__.py mode change 100644 => 100755 airflow/providers/apache/hdfs/hooks/__init__.py mode change 100644 => 100755 airflow/providers/apache/hdfs/hooks/hdfs.py mode change 100644 => 100755 airflow/providers/apache/hdfs/hooks/webhdfs.py mode change 100644 => 100755 airflow/providers/apache/hdfs/sensors/__init__.py mode change 100644 => 100755 airflow/providers/apache/hdfs/sensors/hdfs.py mode change 100644 => 100755 airflow/providers/apache/hdfs/sensors/web_hdfs.py mode change 100644 => 100755 airflow/providers/apache/hive/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/apache/hive/README.md mode change 100644 => 100755 airflow/providers/apache/hive/__init__.py mode change 100644 => 100755 airflow/providers/apache/hive/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/apache/hive/example_dags/example_twitter_README.md mode change 100644 => 100755 airflow/providers/apache/hive/example_dags/example_twitter_dag.py mode change 100644 => 100755 airflow/providers/apache/hive/hooks/__init__.py mode change 100644 => 100755 airflow/providers/apache/hive/hooks/hive.py mode change 100644 => 100755 airflow/providers/apache/hive/operators/__init__.py mode change 100644 => 100755 airflow/providers/apache/hive/operators/hive.py mode change 100644 => 100755 airflow/providers/apache/hive/operators/hive_stats.py mode change 100644 => 100755 airflow/providers/apache/hive/sensors/__init__.py mode change 100644 => 100755 airflow/providers/apache/hive/sensors/hive_partition.py mode change 100644 => 100755 airflow/providers/apache/hive/sensors/metastore_partition.py mode change 100644 => 100755 airflow/providers/apache/hive/sensors/named_hive_partition.py mode change 100644 => 100755 airflow/providers/apache/hive/transfers/__init__.py mode change 100644 => 100755 airflow/providers/apache/hive/transfers/hive_to_mysql.py mode change 100644 => 100755 airflow/providers/apache/hive/transfers/hive_to_samba.py mode change 100644 => 100755 airflow/providers/apache/hive/transfers/mssql_to_hive.py mode change 100644 => 100755 airflow/providers/apache/hive/transfers/mysql_to_hive.py mode change 100644 => 100755 airflow/providers/apache/hive/transfers/s3_to_hive.py mode change 100644 => 100755 airflow/providers/apache/hive/transfers/vertica_to_hive.py mode change 100644 => 100755 airflow/providers/apache/kafka/__init__.py mode change 100644 => 100755 airflow/providers/apache/kafka/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/apache/kafka/example_dags/example_kafka_dag.py mode change 100644 => 100755 airflow/providers/apache/kafka/hooks/__init__.py mode change 100644 => 100755 airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py mode change 100644 => 100755 airflow/providers/apache/kafka/hooks/kafka_producer_hook.py mode change 100644 => 100755 airflow/providers/apache/kafka/sensors/__init__.py mode change 100644 => 100755 airflow/providers/apache/kafka/sensors/kafka_sensor.py mode change 100644 => 100755 airflow/providers/apache/kylin/__init__.py mode change 100644 => 100755 airflow/providers/apache/kylin/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/apache/kylin/example_dags/example_kylin_dag.py mode change 100644 => 100755 airflow/providers/apache/kylin/hooks/__init__.py mode change 100644 => 100755 airflow/providers/apache/kylin/hooks/kylin.py mode change 100644 => 100755 airflow/providers/apache/kylin/operators/__init__.py mode change 100644 => 100755 airflow/providers/apache/kylin/operators/kylin_cube.py mode change 100644 => 100755 airflow/providers/apache/livy/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/apache/livy/README.md mode change 100644 => 100755 airflow/providers/apache/livy/__init__.py mode change 100644 => 100755 airflow/providers/apache/livy/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/apache/livy/example_dags/example_livy.py mode change 100644 => 100755 airflow/providers/apache/livy/hooks/__init__.py mode change 100644 => 100755 airflow/providers/apache/livy/hooks/livy.py mode change 100644 => 100755 airflow/providers/apache/livy/operators/__init__.py mode change 100644 => 100755 airflow/providers/apache/livy/operators/livy.py mode change 100644 => 100755 airflow/providers/apache/livy/sensors/__init__.py mode change 100644 => 100755 airflow/providers/apache/livy/sensors/livy.py mode change 100644 => 100755 airflow/providers/apache/pig/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/apache/pig/README.md mode change 100644 => 100755 airflow/providers/apache/pig/__init__.py mode change 100644 => 100755 airflow/providers/apache/pig/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/apache/pig/example_dags/example_pig.py mode change 100644 => 100755 airflow/providers/apache/pig/hooks/__init__.py mode change 100644 => 100755 airflow/providers/apache/pig/hooks/pig.py mode change 100644 => 100755 airflow/providers/apache/pig/operators/__init__.py mode change 100644 => 100755 airflow/providers/apache/pig/operators/pig.py mode change 100644 => 100755 airflow/providers/apache/pinot/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/apache/pinot/README.md mode change 100644 => 100755 airflow/providers/apache/pinot/__init__.py mode change 100644 => 100755 airflow/providers/apache/pinot/hooks/__init__.py mode change 100644 => 100755 airflow/providers/apache/pinot/hooks/pinot.py mode change 100644 => 100755 airflow/providers/apache/spark/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/apache/spark/README.md mode change 100644 => 100755 airflow/providers/apache/spark/__init__.py mode change 100644 => 100755 airflow/providers/apache/spark/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/apache/spark/example_dags/example_spark_dag.py mode change 100644 => 100755 airflow/providers/apache/spark/hooks/__init__.py mode change 100644 => 100755 airflow/providers/apache/spark/hooks/spark_jdbc.py mode change 100644 => 100755 airflow/providers/apache/spark/hooks/spark_jdbc_script.py mode change 100644 => 100755 airflow/providers/apache/spark/hooks/spark_sql.py mode change 100644 => 100755 airflow/providers/apache/spark/hooks/spark_submit.py mode change 100644 => 100755 airflow/providers/apache/spark/operators/__init__.py mode change 100644 => 100755 airflow/providers/apache/spark/operators/spark_jdbc.py mode change 100644 => 100755 airflow/providers/apache/spark/operators/spark_sql.py mode change 100644 => 100755 airflow/providers/apache/spark/operators/spark_submit.py mode change 100644 => 100755 airflow/providers/apache/sqoop/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/apache/sqoop/README.md mode change 100644 => 100755 airflow/providers/apache/sqoop/__init__.py mode change 100644 => 100755 airflow/providers/apache/sqoop/hooks/__init__.py mode change 100644 => 100755 airflow/providers/apache/sqoop/hooks/sqoop.py mode change 100644 => 100755 airflow/providers/apache/sqoop/operators/__init__.py mode change 100644 => 100755 airflow/providers/apache/sqoop/operators/sqoop.py mode change 100644 => 100755 airflow/providers/celery/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/celery/README.md mode change 100644 => 100755 airflow/providers/celery/__init__.py mode change 100644 => 100755 airflow/providers/celery/sensors/__init__.py mode change 100644 => 100755 airflow/providers/celery/sensors/celery_queue.py mode change 100644 => 100755 airflow/providers/cloudant/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/cloudant/README.md mode change 100644 => 100755 airflow/providers/cloudant/__init__.py mode change 100644 => 100755 airflow/providers/cloudant/hooks/__init__.py mode change 100644 => 100755 airflow/providers/cloudant/hooks/cloudant.py mode change 100644 => 100755 airflow/providers/cncf/__init__.py mode change 100644 => 100755 airflow/providers/cncf/kubernetes/__init__.py mode change 100644 => 100755 airflow/providers/cncf/kubernetes/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/cncf/kubernetes/example_dags/example_kubernetes.py mode change 100644 => 100755 airflow/providers/cncf/kubernetes/example_dags/example_spark_kubernetes.py mode change 100644 => 100755 airflow/providers/cncf/kubernetes/example_dags/example_spark_kubernetes_spark_pi.yaml mode change 100644 => 100755 airflow/providers/cncf/kubernetes/hooks/__init__.py mode change 100644 => 100755 airflow/providers/cncf/kubernetes/hooks/kubernetes.py mode change 100644 => 100755 airflow/providers/cncf/kubernetes/operators/__init__.py mode change 100644 => 100755 airflow/providers/cncf/kubernetes/operators/kubernetes_pod.py mode change 100644 => 100755 airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py mode change 100644 => 100755 airflow/providers/cncf/kubernetes/sensors/__init__.py mode change 100644 => 100755 airflow/providers/cncf/kubernetes/sensors/spark_kubernetes.py mode change 100644 => 100755 airflow/providers/databricks/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/databricks/README.md mode change 100644 => 100755 airflow/providers/databricks/__init__.py mode change 100644 => 100755 airflow/providers/databricks/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/databricks/example_dags/example_databricks.py mode change 100644 => 100755 airflow/providers/databricks/hooks/__init__.py mode change 100644 => 100755 airflow/providers/databricks/hooks/databricks.py mode change 100644 => 100755 airflow/providers/databricks/operators/__init__.py mode change 100644 => 100755 airflow/providers/databricks/operators/databricks.py mode change 100644 => 100755 airflow/providers/datadog/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/datadog/README.md mode change 100644 => 100755 airflow/providers/datadog/__init__.py mode change 100644 => 100755 airflow/providers/datadog/hooks/__init__.py mode change 100644 => 100755 airflow/providers/datadog/hooks/datadog.py mode change 100644 => 100755 airflow/providers/datadog/sensors/__init__.py mode change 100644 => 100755 airflow/providers/datadog/sensors/datadog.py mode change 100644 => 100755 airflow/providers/dependencies.json mode change 100644 => 100755 airflow/providers/dingding/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/dingding/README.md mode change 100644 => 100755 airflow/providers/dingding/__init__.py mode change 100644 => 100755 airflow/providers/dingding/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/dingding/example_dags/example_dingding.py mode change 100644 => 100755 airflow/providers/dingding/hooks/__init__.py mode change 100644 => 100755 airflow/providers/dingding/hooks/dingding.py mode change 100644 => 100755 airflow/providers/dingding/operators/__init__.py mode change 100644 => 100755 airflow/providers/dingding/operators/dingding.py mode change 100644 => 100755 airflow/providers/discord/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/discord/README.md mode change 100644 => 100755 airflow/providers/discord/__init__.py mode change 100644 => 100755 airflow/providers/discord/hooks/__init__.py mode change 100644 => 100755 airflow/providers/discord/hooks/discord_webhook.py mode change 100644 => 100755 airflow/providers/discord/operators/__init__.py mode change 100644 => 100755 airflow/providers/discord/operators/discord_webhook.py mode change 100644 => 100755 airflow/providers/docker/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/docker/README.md mode change 100644 => 100755 airflow/providers/docker/__init__.py mode change 100644 => 100755 airflow/providers/docker/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/docker/example_dags/example_docker.py mode change 100644 => 100755 airflow/providers/docker/example_dags/example_docker_copy_data.py mode change 100644 => 100755 airflow/providers/docker/example_dags/example_docker_swarm.py mode change 100644 => 100755 airflow/providers/docker/hooks/__init__.py mode change 100644 => 100755 airflow/providers/docker/hooks/docker.py mode change 100644 => 100755 airflow/providers/docker/operators/__init__.py mode change 100644 => 100755 airflow/providers/docker/operators/docker.py mode change 100644 => 100755 airflow/providers/docker/operators/docker_swarm.py mode change 100644 => 100755 airflow/providers/elasticsearch/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/elasticsearch/README.md mode change 100644 => 100755 airflow/providers/elasticsearch/__init__.py mode change 100644 => 100755 airflow/providers/elasticsearch/hooks/__init__.py mode change 100644 => 100755 airflow/providers/elasticsearch/hooks/elasticsearch.py mode change 100644 => 100755 airflow/providers/elasticsearch/log/__init__.py mode change 100644 => 100755 airflow/providers/elasticsearch/log/es_task_handler.py mode change 100644 => 100755 airflow/providers/exasol/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/exasol/README.md mode change 100644 => 100755 airflow/providers/exasol/__init__.py mode change 100644 => 100755 airflow/providers/exasol/hooks/__init__.py mode change 100644 => 100755 airflow/providers/exasol/hooks/exasol.py mode change 100644 => 100755 airflow/providers/exasol/operators/__init__.py mode change 100644 => 100755 airflow/providers/exasol/operators/exasol.py mode change 100644 => 100755 airflow/providers/facebook/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/facebook/README.md mode change 100644 => 100755 airflow/providers/facebook/__init__.py mode change 100644 => 100755 airflow/providers/facebook/ads/__init__.py mode change 100644 => 100755 airflow/providers/facebook/ads/hooks/__init__.py mode change 100644 => 100755 airflow/providers/facebook/ads/hooks/ads.py mode change 100644 => 100755 airflow/providers/ftp/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/ftp/README.md mode change 100644 => 100755 airflow/providers/ftp/__init__.py mode change 100644 => 100755 airflow/providers/ftp/hooks/__init__.py mode change 100644 => 100755 airflow/providers/ftp/hooks/ftp.py mode change 100644 => 100755 airflow/providers/ftp/sensors/__init__.py mode change 100644 => 100755 airflow/providers/ftp/sensors/ftp.py mode change 100644 => 100755 airflow/providers/google/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/google/README.md mode change 100644 => 100755 airflow/providers/google/__init__.py mode change 100644 => 100755 airflow/providers/google/ads/__init__.py mode change 100644 => 100755 airflow/providers/google/ads/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/google/ads/example_dags/example_ads.py mode change 100644 => 100755 airflow/providers/google/ads/hooks/__init__.py mode change 100644 => 100755 airflow/providers/google/ads/hooks/ads.py mode change 100644 => 100755 airflow/providers/google/ads/operators/__init__.py mode change 100644 => 100755 airflow/providers/google/ads/operators/ads.py mode change 100644 => 100755 airflow/providers/google/ads/transfers/__init__.py mode change 100644 => 100755 airflow/providers/google/ads/transfers/ads_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/__init__.py mode change 100644 => 100755 airflow/providers/google/cloud/_internal_client/__init__.py mode change 100644 => 100755 airflow/providers/google/cloud/_internal_client/secret_manager_client.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_automl_nl_text_extraction.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_automl_tables.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_automl_translation.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_automl_vision_classification.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_bigquery_dts.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_bigquery_operations.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_bigquery_queries.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_bigquery_query.sql mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_bigquery_sensors.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_bigquery_to_bigquery.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_bigquery_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_bigquery_transfer.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_bigtable.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_cloud_build.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_cloud_build.yaml mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_cloud_memorystore.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_cloud_sql.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_compute.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_compute_igm.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_datacatalog.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_dataflow.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_datafusion.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_dataprep.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_dataproc.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_datastore.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_dlp.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_functions.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_gcs_to_bigquery.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_gcs_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_gcs_to_sftp.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_kubernetes_engine.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_life_sciences.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_local_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_mlengine.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_natural_language.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_pubsub.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_sheets_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_spanner.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_spanner.sql mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_speech_to_text.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_stackdriver.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_tasks.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_text_to_speech.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_translate.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_translate_speech.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_video_intelligence.py mode change 100644 => 100755 airflow/providers/google/cloud/example_dags/example_vision.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/__init__.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/automl.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/bigquery.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/bigquery_dts.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/bigtable.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/cloud_build.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/cloud_memorystore.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/cloud_sql.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/compute.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/datacatalog.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/dataflow.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/datafusion.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/dataprep.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/dataproc.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/datastore.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/dlp.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/functions.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/gdm.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/kms.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/kubernetes_engine.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/life_sciences.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/mlengine.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/natural_language.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/pubsub.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/secret_manager.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/spanner.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/speech_to_text.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/stackdriver.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/tasks.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/text_to_speech.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/translate.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/video_intelligence.py mode change 100644 => 100755 airflow/providers/google/cloud/hooks/vision.py mode change 100644 => 100755 airflow/providers/google/cloud/log/__init__.py mode change 100644 => 100755 airflow/providers/google/cloud/log/gcs_task_handler.py mode change 100644 => 100755 airflow/providers/google/cloud/log/stackdriver_task_handler.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/__init__.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/automl.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/bigquery.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/bigquery_dts.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/bigtable.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/cloud_build.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/cloud_memorystore.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/cloud_sql.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/cloud_storage_transfer_service.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/compute.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/datacatalog.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/dataflow.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/datafusion.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/dataprep.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/dataproc.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/datastore.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/dlp.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/functions.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/kubernetes_engine.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/life_sciences.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/mlengine.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/natural_language.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/pubsub.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/spanner.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/speech_to_text.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/stackdriver.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/tasks.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/text_to_speech.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/translate.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/translate_speech.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/video_intelligence.py mode change 100644 => 100755 airflow/providers/google/cloud/operators/vision.py mode change 100644 => 100755 airflow/providers/google/cloud/secrets/__init__.py mode change 100644 => 100755 airflow/providers/google/cloud/secrets/secret_manager.py mode change 100644 => 100755 airflow/providers/google/cloud/sensors/__init__.py mode change 100644 => 100755 airflow/providers/google/cloud/sensors/bigquery.py mode change 100644 => 100755 airflow/providers/google/cloud/sensors/bigquery_dts.py mode change 100644 => 100755 airflow/providers/google/cloud/sensors/bigtable.py mode change 100644 => 100755 airflow/providers/google/cloud/sensors/cloud_storage_transfer_service.py mode change 100644 => 100755 airflow/providers/google/cloud/sensors/gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/sensors/pubsub.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/__init__.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/adls_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/bigquery_to_bigquery.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/bigquery_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/bigquery_to_mysql.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/cassandra_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/facebook_ads_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/gcs_to_bigquery.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/gcs_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/gcs_to_local.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/gcs_to_sftp.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/local_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/mssql_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/mysql_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/postgres_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/presto_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/s3_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/sftp_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/sheets_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/transfers/sql_to_gcs.py mode change 100644 => 100755 airflow/providers/google/cloud/utils/__init__.py mode change 100644 => 100755 airflow/providers/google/cloud/utils/credentials_provider.py mode change 100644 => 100755 airflow/providers/google/cloud/utils/field_sanitizer.py mode change 100644 => 100755 airflow/providers/google/cloud/utils/field_validator.py mode change 100644 => 100755 airflow/providers/google/cloud/utils/mlengine_operator_utils.py mode change 100644 => 100755 airflow/providers/google/cloud/utils/mlengine_prediction_summary.py mode change 100644 => 100755 airflow/providers/google/common/__init__.py mode change 100644 => 100755 airflow/providers/google/common/auth_backend/__init__.py mode change 100644 => 100755 airflow/providers/google/common/auth_backend/google_openid.py mode change 100644 => 100755 airflow/providers/google/common/hooks/__init__.py mode change 100644 => 100755 airflow/providers/google/common/hooks/base_google.py mode change 100644 => 100755 airflow/providers/google/common/hooks/discovery_api.py mode change 100644 => 100755 airflow/providers/google/common/utils/__init__.py mode change 100644 => 100755 airflow/providers/google/common/utils/id_token_credentials.py mode change 100644 => 100755 airflow/providers/google/config_templates/config.yml mode change 100644 => 100755 airflow/providers/google/config_templates/default_config.cfg mode change 100644 => 100755 airflow/providers/google/firebase/__init__.py mode change 100644 => 100755 airflow/providers/google/firebase/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/google/firebase/example_dags/example_firestore.py mode change 100644 => 100755 airflow/providers/google/firebase/hooks/__init__.py mode change 100644 => 100755 airflow/providers/google/firebase/hooks/firestore.py mode change 100644 => 100755 airflow/providers/google/firebase/operators/__init__.py mode change 100644 => 100755 airflow/providers/google/firebase/operators/firestore.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/__init__.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/example_dags/example_analytics.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/example_dags/example_campaign_manager.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/example_dags/example_display_video.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/example_dags/example_search_ads.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/hooks/__init__.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/hooks/analytics.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/hooks/campaign_manager.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/hooks/display_video.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/hooks/search_ads.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/operators/__init__.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/operators/analytics.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/operators/campaign_manager.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/operators/display_video.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/operators/search_ads.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/sensors/__init__.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/sensors/campaign_manager.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/sensors/display_video.py mode change 100644 => 100755 airflow/providers/google/marketing_platform/sensors/search_ads.py mode change 100644 => 100755 airflow/providers/google/suite/__init__.py mode change 100644 => 100755 airflow/providers/google/suite/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/google/suite/example_dags/example_gcs_to_gdrive.py mode change 100644 => 100755 airflow/providers/google/suite/example_dags/example_gcs_to_sheets.py mode change 100644 => 100755 airflow/providers/google/suite/example_dags/example_sheets.py mode change 100644 => 100755 airflow/providers/google/suite/hooks/__init__.py mode change 100644 => 100755 airflow/providers/google/suite/hooks/drive.py mode change 100644 => 100755 airflow/providers/google/suite/hooks/sheets.py mode change 100644 => 100755 airflow/providers/google/suite/operators/__init__.py mode change 100644 => 100755 airflow/providers/google/suite/operators/sheets.py mode change 100644 => 100755 airflow/providers/google/suite/transfers/__init__.py mode change 100644 => 100755 airflow/providers/google/suite/transfers/gcs_to_gdrive.py mode change 100644 => 100755 airflow/providers/google/suite/transfers/gcs_to_sheets.py mode change 100644 => 100755 airflow/providers/grpc/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/grpc/README.md mode change 100644 => 100755 airflow/providers/grpc/__init__.py mode change 100644 => 100755 airflow/providers/grpc/hooks/__init__.py mode change 100644 => 100755 airflow/providers/grpc/hooks/grpc.py mode change 100644 => 100755 airflow/providers/grpc/operators/__init__.py mode change 100644 => 100755 airflow/providers/grpc/operators/grpc.py mode change 100644 => 100755 airflow/providers/hashicorp/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/hashicorp/README.md mode change 100644 => 100755 airflow/providers/hashicorp/__init__.py mode change 100644 => 100755 airflow/providers/hashicorp/_internal_client/__init__.py mode change 100644 => 100755 airflow/providers/hashicorp/_internal_client/vault_client.py mode change 100644 => 100755 airflow/providers/hashicorp/hooks/__init__.py mode change 100644 => 100755 airflow/providers/hashicorp/hooks/vault.py mode change 100644 => 100755 airflow/providers/hashicorp/secrets/__init__.py mode change 100644 => 100755 airflow/providers/hashicorp/secrets/vault.py mode change 100644 => 100755 airflow/providers/http/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/http/README.md mode change 100644 => 100755 airflow/providers/http/__init__.py mode change 100644 => 100755 airflow/providers/http/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/http/example_dags/example_http.py mode change 100644 => 100755 airflow/providers/http/hooks/__init__.py mode change 100644 => 100755 airflow/providers/http/hooks/http.py mode change 100644 => 100755 airflow/providers/http/operators/__init__.py mode change 100644 => 100755 airflow/providers/http/operators/http.py mode change 100644 => 100755 airflow/providers/http/sensors/__init__.py mode change 100644 => 100755 airflow/providers/http/sensors/http.py mode change 100644 => 100755 airflow/providers/imap/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/imap/README.md mode change 100644 => 100755 airflow/providers/imap/__init__.py mode change 100644 => 100755 airflow/providers/imap/hooks/__init__.py mode change 100644 => 100755 airflow/providers/imap/hooks/imap.py mode change 100644 => 100755 airflow/providers/imap/sensors/__init__.py mode change 100644 => 100755 airflow/providers/imap/sensors/imap_attachment.py mode change 100644 => 100755 airflow/providers/jdbc/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/jdbc/README.md mode change 100644 => 100755 airflow/providers/jdbc/__init__.py mode change 100644 => 100755 airflow/providers/jdbc/hooks/__init__.py mode change 100644 => 100755 airflow/providers/jdbc/hooks/jdbc.py mode change 100644 => 100755 airflow/providers/jdbc/operators/__init__.py mode change 100644 => 100755 airflow/providers/jdbc/operators/jdbc.py mode change 100644 => 100755 airflow/providers/jenkins/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/jenkins/README.md mode change 100644 => 100755 airflow/providers/jenkins/__init__.py mode change 100644 => 100755 airflow/providers/jenkins/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/jenkins/example_dags/example_jenkins_job_trigger.py mode change 100644 => 100755 airflow/providers/jenkins/hooks/__init__.py mode change 100644 => 100755 airflow/providers/jenkins/hooks/jenkins.py mode change 100644 => 100755 airflow/providers/jenkins/operators/__init__.py mode change 100644 => 100755 airflow/providers/jenkins/operators/jenkins_job_trigger.py mode change 100644 => 100755 airflow/providers/jira/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/jira/README.md mode change 100644 => 100755 airflow/providers/jira/__init__.py mode change 100644 => 100755 airflow/providers/jira/hooks/__init__.py mode change 100644 => 100755 airflow/providers/jira/hooks/jira.py mode change 100644 => 100755 airflow/providers/jira/operators/__init__.py mode change 100644 => 100755 airflow/providers/jira/operators/jira.py mode change 100644 => 100755 airflow/providers/jira/sensors/__init__.py mode change 100644 => 100755 airflow/providers/jira/sensors/jira.py mode change 100644 => 100755 airflow/providers/microsoft/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/azure/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/microsoft/azure/README.md mode change 100644 => 100755 airflow/providers/microsoft/azure/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/azure/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/azure/example_dags/example_azure_container_instances.py mode change 100644 => 100755 airflow/providers/microsoft/azure/example_dags/example_azure_cosmosdb.py mode change 100644 => 100755 airflow/providers/microsoft/azure/hooks/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/azure/hooks/adx.py mode change 100644 => 100755 airflow/providers/microsoft/azure/hooks/azure_batch.py mode change 100644 => 100755 airflow/providers/microsoft/azure/hooks/azure_container_instance.py mode change 100644 => 100755 airflow/providers/microsoft/azure/hooks/azure_container_registry.py mode change 100644 => 100755 airflow/providers/microsoft/azure/hooks/azure_container_volume.py mode change 100644 => 100755 airflow/providers/microsoft/azure/hooks/azure_cosmos.py mode change 100644 => 100755 airflow/providers/microsoft/azure/hooks/azure_data_lake.py mode change 100644 => 100755 airflow/providers/microsoft/azure/hooks/azure_fileshare.py mode change 100644 => 100755 airflow/providers/microsoft/azure/hooks/base_azure.py mode change 100644 => 100755 airflow/providers/microsoft/azure/hooks/wasb.py mode change 100644 => 100755 airflow/providers/microsoft/azure/log/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/azure/log/wasb_task_handler.py mode change 100644 => 100755 airflow/providers/microsoft/azure/operators/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/azure/operators/adls_list.py mode change 100644 => 100755 airflow/providers/microsoft/azure/operators/adx.py mode change 100644 => 100755 airflow/providers/microsoft/azure/operators/azure_batch.py mode change 100644 => 100755 airflow/providers/microsoft/azure/operators/azure_container_instances.py mode change 100644 => 100755 airflow/providers/microsoft/azure/operators/azure_cosmos.py mode change 100644 => 100755 airflow/providers/microsoft/azure/operators/wasb_delete_blob.py mode change 100644 => 100755 airflow/providers/microsoft/azure/sensors/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/azure/sensors/azure_cosmos.py mode change 100644 => 100755 airflow/providers/microsoft/azure/sensors/wasb.py mode change 100644 => 100755 airflow/providers/microsoft/azure/transfers/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/azure/transfers/file_to_wasb.py mode change 100644 => 100755 airflow/providers/microsoft/azure/transfers/oracle_to_azure_data_lake.py mode change 100644 => 100755 airflow/providers/microsoft/mssql/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/microsoft/mssql/README.md mode change 100644 => 100755 airflow/providers/microsoft/mssql/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/mssql/hooks/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/mssql/hooks/mssql.py mode change 100644 => 100755 airflow/providers/microsoft/mssql/operators/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/mssql/operators/mssql.py mode change 100644 => 100755 airflow/providers/microsoft/winrm/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/microsoft/winrm/README.md mode change 100644 => 100755 airflow/providers/microsoft/winrm/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/winrm/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/winrm/example_dags/example_winrm.py mode change 100644 => 100755 airflow/providers/microsoft/winrm/hooks/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/winrm/hooks/winrm.py mode change 100644 => 100755 airflow/providers/microsoft/winrm/operators/__init__.py mode change 100644 => 100755 airflow/providers/microsoft/winrm/operators/winrm.py mode change 100644 => 100755 airflow/providers/mongo/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/mongo/README.md mode change 100644 => 100755 airflow/providers/mongo/__init__.py mode change 100644 => 100755 airflow/providers/mongo/hooks/__init__.py mode change 100644 => 100755 airflow/providers/mongo/hooks/mongo.py mode change 100644 => 100755 airflow/providers/mongo/sensors/__init__.py mode change 100644 => 100755 airflow/providers/mongo/sensors/mongo.py mode change 100644 => 100755 airflow/providers/mysql/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/mysql/README.md mode change 100644 => 100755 airflow/providers/mysql/__init__.py mode change 100644 => 100755 airflow/providers/mysql/hooks/__init__.py mode change 100644 => 100755 airflow/providers/mysql/hooks/mysql.py mode change 100644 => 100755 airflow/providers/mysql/operators/__init__.py mode change 100644 => 100755 airflow/providers/mysql/operators/mysql.py mode change 100644 => 100755 airflow/providers/mysql/transfers/__init__.py mode change 100644 => 100755 airflow/providers/mysql/transfers/presto_to_mysql.py mode change 100644 => 100755 airflow/providers/mysql/transfers/s3_to_mysql.py mode change 100644 => 100755 airflow/providers/mysql/transfers/vertica_to_mysql.py mode change 100644 => 100755 airflow/providers/odbc/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/odbc/README.md mode change 100644 => 100755 airflow/providers/odbc/__init__.py mode change 100644 => 100755 airflow/providers/odbc/hooks/__init__.py mode change 100644 => 100755 airflow/providers/odbc/hooks/odbc.py mode change 100644 => 100755 airflow/providers/openfaas/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/openfaas/README.md mode change 100644 => 100755 airflow/providers/openfaas/__init__.py mode change 100644 => 100755 airflow/providers/openfaas/hooks/__init__.py mode change 100644 => 100755 airflow/providers/openfaas/hooks/openfaas.py mode change 100644 => 100755 airflow/providers/opsgenie/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/opsgenie/README.md mode change 100644 => 100755 airflow/providers/opsgenie/__init__.py mode change 100644 => 100755 airflow/providers/opsgenie/hooks/__init__.py mode change 100644 => 100755 airflow/providers/opsgenie/hooks/opsgenie_alert.py mode change 100644 => 100755 airflow/providers/opsgenie/operators/__init__.py mode change 100644 => 100755 airflow/providers/opsgenie/operators/opsgenie_alert.py mode change 100644 => 100755 airflow/providers/oracle/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/oracle/README.md mode change 100644 => 100755 airflow/providers/oracle/__init__.py mode change 100644 => 100755 airflow/providers/oracle/hooks/__init__.py mode change 100644 => 100755 airflow/providers/oracle/hooks/oracle.py mode change 100644 => 100755 airflow/providers/oracle/operators/__init__.py mode change 100644 => 100755 airflow/providers/oracle/operators/oracle.py mode change 100644 => 100755 airflow/providers/oracle/transfers/__init__.py mode change 100644 => 100755 airflow/providers/oracle/transfers/oracle_to_oracle.py mode change 100644 => 100755 airflow/providers/pagerduty/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/pagerduty/README.md mode change 100644 => 100755 airflow/providers/pagerduty/__init__.py mode change 100644 => 100755 airflow/providers/pagerduty/hooks/__init__.py mode change 100644 => 100755 airflow/providers/pagerduty/hooks/pagerduty.py mode change 100644 => 100755 airflow/providers/papermill/ADDITIONAL_INFO.md mode change 100644 => 100755 airflow/providers/papermill/__init__.py mode change 100644 => 100755 airflow/providers/papermill/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/papermill/example_dags/example_papermill.py mode change 100644 => 100755 airflow/providers/papermill/operators/__init__.py mode change 100644 => 100755 airflow/providers/papermill/operators/papermill.py mode change 100644 => 100755 airflow/providers/postgres/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/postgres/README.md mode change 100644 => 100755 airflow/providers/postgres/__init__.py mode change 100644 => 100755 airflow/providers/postgres/hooks/__init__.py mode change 100644 => 100755 airflow/providers/postgres/hooks/postgres.py mode change 100644 => 100755 airflow/providers/postgres/operators/__init__.py mode change 100644 => 100755 airflow/providers/postgres/operators/postgres.py mode change 100644 => 100755 airflow/providers/presto/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/presto/README.md mode change 100644 => 100755 airflow/providers/presto/__init__.py mode change 100644 => 100755 airflow/providers/presto/hooks/__init__.py mode change 100644 => 100755 airflow/providers/presto/hooks/presto.py mode change 100644 => 100755 airflow/providers/qubole/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/qubole/README.md mode change 100644 => 100755 airflow/providers/qubole/__init__.py mode change 100644 => 100755 airflow/providers/qubole/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/qubole/example_dags/example_qubole.py mode change 100644 => 100755 airflow/providers/qubole/hooks/__init__.py mode change 100644 => 100755 airflow/providers/qubole/hooks/qubole.py mode change 100644 => 100755 airflow/providers/qubole/hooks/qubole_check.py mode change 100644 => 100755 airflow/providers/qubole/operators/__init__.py mode change 100644 => 100755 airflow/providers/qubole/operators/qubole.py mode change 100644 => 100755 airflow/providers/qubole/operators/qubole_check.py mode change 100644 => 100755 airflow/providers/qubole/sensors/__init__.py mode change 100644 => 100755 airflow/providers/qubole/sensors/qubole.py mode change 100644 => 100755 airflow/providers/redis/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/redis/README.md mode change 100644 => 100755 airflow/providers/redis/__init__.py mode change 100644 => 100755 airflow/providers/redis/hooks/__init__.py mode change 100644 => 100755 airflow/providers/redis/hooks/redis.py mode change 100644 => 100755 airflow/providers/redis/operators/__init__.py mode change 100644 => 100755 airflow/providers/redis/operators/redis_publish.py mode change 100644 => 100755 airflow/providers/redis/sensors/__init__.py mode change 100644 => 100755 airflow/providers/redis/sensors/redis_key.py mode change 100644 => 100755 airflow/providers/redis/sensors/redis_pub_sub.py mode change 100644 => 100755 airflow/providers/salesforce/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/salesforce/README.md mode change 100644 => 100755 airflow/providers/salesforce/__init__.py mode change 100644 => 100755 airflow/providers/salesforce/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/salesforce/example_dags/example_tableau_refresh_workbook.py mode change 100644 => 100755 airflow/providers/salesforce/hooks/__init__.py mode change 100644 => 100755 airflow/providers/salesforce/hooks/salesforce.py mode change 100644 => 100755 airflow/providers/salesforce/hooks/tableau.py mode change 100644 => 100755 airflow/providers/salesforce/operators/__init__.py mode change 100644 => 100755 airflow/providers/salesforce/operators/tableau_refresh_workbook.py mode change 100644 => 100755 airflow/providers/salesforce/sensors/__init__.py mode change 100644 => 100755 airflow/providers/salesforce/sensors/tableau_job_status.py mode change 100644 => 100755 airflow/providers/samba/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/samba/README.md mode change 100644 => 100755 airflow/providers/samba/__init__.py mode change 100644 => 100755 airflow/providers/samba/hooks/__init__.py mode change 100644 => 100755 airflow/providers/samba/hooks/samba.py mode change 100644 => 100755 airflow/providers/segment/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/segment/README.md mode change 100644 => 100755 airflow/providers/segment/__init__.py mode change 100644 => 100755 airflow/providers/segment/hooks/__init__.py mode change 100644 => 100755 airflow/providers/segment/hooks/segment.py mode change 100644 => 100755 airflow/providers/segment/operators/__init__.py mode change 100644 => 100755 airflow/providers/segment/operators/segment_track_event.py mode change 100644 => 100755 airflow/providers/sendgrid/__init__.py mode change 100644 => 100755 airflow/providers/sendgrid/utils/__init__.py mode change 100644 => 100755 airflow/providers/sendgrid/utils/emailer.py mode change 100644 => 100755 airflow/providers/sftp/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/sftp/README.md mode change 100644 => 100755 airflow/providers/sftp/__init__.py mode change 100644 => 100755 airflow/providers/sftp/hooks/__init__.py mode change 100644 => 100755 airflow/providers/sftp/hooks/sftp.py mode change 100644 => 100755 airflow/providers/sftp/operators/__init__.py mode change 100644 => 100755 airflow/providers/sftp/operators/sftp.py mode change 100644 => 100755 airflow/providers/sftp/sensors/__init__.py mode change 100644 => 100755 airflow/providers/sftp/sensors/sftp.py mode change 100644 => 100755 airflow/providers/singularity/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/singularity/README.md mode change 100644 => 100755 airflow/providers/singularity/__init__.py mode change 100644 => 100755 airflow/providers/singularity/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/singularity/example_dags/example_singularity.py mode change 100644 => 100755 airflow/providers/singularity/operators/__init__.py mode change 100644 => 100755 airflow/providers/singularity/operators/singularity.py mode change 100644 => 100755 airflow/providers/slack/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/slack/README.md mode change 100644 => 100755 airflow/providers/slack/__init__.py mode change 100644 => 100755 airflow/providers/slack/hooks/__init__.py mode change 100644 => 100755 airflow/providers/slack/hooks/slack.py mode change 100644 => 100755 airflow/providers/slack/hooks/slack_webhook.py mode change 100644 => 100755 airflow/providers/slack/operators/__init__.py mode change 100644 => 100755 airflow/providers/slack/operators/slack.py mode change 100644 => 100755 airflow/providers/slack/operators/slack_webhook.py mode change 100644 => 100755 airflow/providers/snowflake/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/snowflake/README.md mode change 100644 => 100755 airflow/providers/snowflake/__init__.py mode change 100644 => 100755 airflow/providers/snowflake/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/snowflake/example_dags/example_snowflake.py mode change 100644 => 100755 airflow/providers/snowflake/hooks/__init__.py mode change 100644 => 100755 airflow/providers/snowflake/hooks/snowflake.py mode change 100644 => 100755 airflow/providers/snowflake/operators/__init__.py mode change 100644 => 100755 airflow/providers/snowflake/operators/snowflake.py mode change 100644 => 100755 airflow/providers/snowflake/transfers/__init__.py mode change 100644 => 100755 airflow/providers/snowflake/transfers/s3_to_snowflake.py mode change 100644 => 100755 airflow/providers/snowflake/transfers/snowflake_to_slack.py mode change 100644 => 100755 airflow/providers/sqlite/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/sqlite/README.md mode change 100644 => 100755 airflow/providers/sqlite/__init__.py mode change 100644 => 100755 airflow/providers/sqlite/hooks/__init__.py mode change 100644 => 100755 airflow/providers/sqlite/hooks/sqlite.py mode change 100644 => 100755 airflow/providers/sqlite/operators/__init__.py mode change 100644 => 100755 airflow/providers/sqlite/operators/sqlite.py mode change 100644 => 100755 airflow/providers/ssh/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/ssh/README.md mode change 100644 => 100755 airflow/providers/ssh/__init__.py mode change 100644 => 100755 airflow/providers/ssh/hooks/__init__.py mode change 100644 => 100755 airflow/providers/ssh/hooks/ssh.py mode change 100644 => 100755 airflow/providers/ssh/operators/__init__.py mode change 100644 => 100755 airflow/providers/ssh/operators/ssh.py mode change 100644 => 100755 airflow/providers/vertica/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/vertica/README.md mode change 100644 => 100755 airflow/providers/vertica/__init__.py mode change 100644 => 100755 airflow/providers/vertica/hooks/__init__.py mode change 100644 => 100755 airflow/providers/vertica/hooks/vertica.py mode change 100644 => 100755 airflow/providers/vertica/operators/__init__.py mode change 100644 => 100755 airflow/providers/vertica/operators/vertica.py mode change 100644 => 100755 airflow/providers/yandex/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/yandex/README.md mode change 100644 => 100755 airflow/providers/yandex/__init__.py mode change 100644 => 100755 airflow/providers/yandex/example_dags/__init__.py mode change 100644 => 100755 airflow/providers/yandex/example_dags/example_yandexcloud_dataproc.py mode change 100644 => 100755 airflow/providers/yandex/hooks/__init__.py mode change 100644 => 100755 airflow/providers/yandex/hooks/yandex.py mode change 100644 => 100755 airflow/providers/yandex/hooks/yandexcloud_dataproc.py mode change 100644 => 100755 airflow/providers/yandex/operators/__init__.py mode change 100644 => 100755 airflow/providers/yandex/operators/yandexcloud_dataproc.py mode change 100644 => 100755 airflow/providers/zendesk/PROVIDERS_CHANGES_2020.06.24.md mode change 100644 => 100755 airflow/providers/zendesk/README.md mode change 100644 => 100755 airflow/providers/zendesk/__init__.py mode change 100644 => 100755 airflow/providers/zendesk/hooks/__init__.py mode change 100644 => 100755 airflow/providers/zendesk/hooks/zendesk.py mode change 100644 => 100755 airflow/py.typed mode change 100644 => 100755 airflow/secrets/__init__.py mode change 100644 => 100755 airflow/secrets/base_secrets.py mode change 100644 => 100755 airflow/secrets/environment_variables.py mode change 100644 => 100755 airflow/secrets/local_filesystem.py mode change 100644 => 100755 airflow/secrets/metastore.py mode change 100644 => 100755 airflow/security/__init__.py mode change 100644 => 100755 airflow/security/kerberos.py mode change 100644 => 100755 airflow/security/utils.py mode change 100644 => 100755 airflow/sensors/__init__.py mode change 100644 => 100755 airflow/sensors/base_sensor_operator.py mode change 100644 => 100755 airflow/sensors/bash.py mode change 100644 => 100755 airflow/sensors/date_time_sensor.py mode change 100644 => 100755 airflow/sensors/external_task_sensor.py mode change 100644 => 100755 airflow/sensors/filesystem.py mode change 100644 => 100755 airflow/sensors/hdfs_sensor.py mode change 100644 => 100755 airflow/sensors/hive_partition_sensor.py mode change 100644 => 100755 airflow/sensors/http_sensor.py mode change 100644 => 100755 airflow/sensors/metastore_partition_sensor.py mode change 100644 => 100755 airflow/sensors/named_hive_partition_sensor.py mode change 100644 => 100755 airflow/sensors/python.py mode change 100644 => 100755 airflow/sensors/s3_key_sensor.py mode change 100644 => 100755 airflow/sensors/s3_prefix_sensor.py mode change 100644 => 100755 airflow/sensors/sql_sensor.py mode change 100644 => 100755 airflow/sensors/time_delta_sensor.py mode change 100644 => 100755 airflow/sensors/time_sensor.py mode change 100644 => 100755 airflow/sensors/web_hdfs_sensor.py mode change 100644 => 100755 airflow/sensors/weekday_sensor.py mode change 100644 => 100755 airflow/sentry.py mode change 100644 => 100755 airflow/serialization/__init__.py mode change 100644 => 100755 airflow/serialization/enums.py mode change 100644 => 100755 airflow/serialization/helpers.py mode change 100644 => 100755 airflow/serialization/json_schema.py mode change 100644 => 100755 airflow/serialization/schema.json mode change 100644 => 100755 airflow/serialization/serialized_objects.py mode change 100644 => 100755 airflow/settings.py mode change 100644 => 100755 airflow/stats.py mode change 100644 => 100755 airflow/task/__init__.py mode change 100644 => 100755 airflow/task/task_runner/__init__.py mode change 100644 => 100755 airflow/task/task_runner/base_task_runner.py mode change 100644 => 100755 airflow/task/task_runner/cgroup_task_runner.py mode change 100644 => 100755 airflow/task/task_runner/standard_task_runner.py mode change 100644 => 100755 airflow/ti_deps/__init__.py mode change 100644 => 100755 airflow/ti_deps/dep_context.py mode change 100644 => 100755 airflow/ti_deps/dependencies_deps.py mode change 100644 => 100755 airflow/ti_deps/dependencies_states.py mode change 100644 => 100755 airflow/ti_deps/deps/__init__.py mode change 100644 => 100755 airflow/ti_deps/deps/base_ti_dep.py mode change 100644 => 100755 airflow/ti_deps/deps/dag_ti_slots_available_dep.py mode change 100644 => 100755 airflow/ti_deps/deps/dag_unpaused_dep.py mode change 100644 => 100755 airflow/ti_deps/deps/dagrun_exists_dep.py mode change 100644 => 100755 airflow/ti_deps/deps/dagrun_id_dep.py mode change 100644 => 100755 airflow/ti_deps/deps/exec_date_after_start_date_dep.py mode change 100644 => 100755 airflow/ti_deps/deps/not_in_retry_period_dep.py mode change 100644 => 100755 airflow/ti_deps/deps/not_previously_skipped_dep.py mode change 100644 => 100755 airflow/ti_deps/deps/pool_slots_available_dep.py mode change 100644 => 100755 airflow/ti_deps/deps/prev_dagrun_dep.py mode change 100644 => 100755 airflow/ti_deps/deps/ready_to_reschedule.py mode change 100644 => 100755 airflow/ti_deps/deps/runnable_exec_date_dep.py mode change 100644 => 100755 airflow/ti_deps/deps/task_concurrency_dep.py mode change 100644 => 100755 airflow/ti_deps/deps/task_not_running_dep.py mode change 100644 => 100755 airflow/ti_deps/deps/trigger_rule_dep.py mode change 100644 => 100755 airflow/ti_deps/deps/valid_state_dep.py mode change 100644 => 100755 airflow/typing_compat.py mode change 100644 => 100755 airflow/utils/__init__.py mode change 100644 => 100755 airflow/utils/asciiart.py mode change 100644 => 100755 airflow/utils/cli.py mode change 100644 => 100755 airflow/utils/cli_action_loggers.py mode change 100644 => 100755 airflow/utils/code_utils.py mode change 100644 => 100755 airflow/utils/compression.py mode change 100644 => 100755 airflow/utils/configuration.py mode change 100644 => 100755 airflow/utils/dag_cycle_tester.py mode change 100644 => 100755 airflow/utils/dag_processing.py mode change 100644 => 100755 airflow/utils/dates.py mode change 100644 => 100755 airflow/utils/db.py mode change 100644 => 100755 airflow/utils/decorators.py mode change 100644 => 100755 airflow/utils/docs.py mode change 100644 => 100755 airflow/utils/dot_renderer.py mode change 100644 => 100755 airflow/utils/email.py mode change 100644 => 100755 airflow/utils/file.py mode change 100644 => 100755 airflow/utils/helpers.py mode change 100644 => 100755 airflow/utils/json.py mode change 100644 => 100755 airflow/utils/log/__init__.py mode change 100644 => 100755 airflow/utils/log/cloudwatch_task_handler.py mode change 100644 => 100755 airflow/utils/log/colored_log.py mode change 100644 => 100755 airflow/utils/log/es_task_handler.py mode change 100644 => 100755 airflow/utils/log/file_processor_handler.py mode change 100644 => 100755 airflow/utils/log/file_task_handler.py mode change 100644 => 100755 airflow/utils/log/gcs_task_handler.py mode change 100644 => 100755 airflow/utils/log/json_formatter.py mode change 100644 => 100755 airflow/utils/log/log_reader.py mode change 100644 => 100755 airflow/utils/log/logging_mixin.py mode change 100644 => 100755 airflow/utils/log/s3_task_handler.py mode change 100644 => 100755 airflow/utils/log/stackdriver_task_handler.py mode change 100644 => 100755 airflow/utils/log/task_handler_with_custom_formatter.py mode change 100644 => 100755 airflow/utils/log/wasb_task_handler.py mode change 100644 => 100755 airflow/utils/mixins.py mode change 100644 => 100755 airflow/utils/module_loading.py mode change 100644 => 100755 airflow/utils/net.py mode change 100644 => 100755 airflow/utils/operator_helpers.py mode change 100644 => 100755 airflow/utils/operator_resources.py mode change 100644 => 100755 airflow/utils/orm_event_handlers.py mode change 100644 => 100755 airflow/utils/platform.py mode change 100644 => 100755 airflow/utils/process_utils.py mode change 100644 => 100755 airflow/utils/python_virtualenv.py mode change 100644 => 100755 airflow/utils/python_virtualenv_script.jinja2 mode change 100644 => 100755 airflow/utils/serve_logs.py mode change 100644 => 100755 airflow/utils/session.py mode change 100644 => 100755 airflow/utils/sqlalchemy.py mode change 100644 => 100755 airflow/utils/state.py mode change 100644 => 100755 airflow/utils/strings.py mode change 100644 => 100755 airflow/utils/timeout.py mode change 100644 => 100755 airflow/utils/timezone.py mode change 100644 => 100755 airflow/utils/trigger_rule.py mode change 100644 => 100755 airflow/utils/types.py mode change 100644 => 100755 airflow/utils/weekday.py mode change 100644 => 100755 airflow/utils/weight_rule.py mode change 100644 => 100755 airflow/version.py mode change 100644 => 100755 airflow/www/.eslintignore mode change 100644 => 100755 airflow/www/.eslintrc mode change 100644 => 100755 airflow/www/.stylelintignore mode change 100644 => 100755 airflow/www/.stylelintrc mode change 100644 => 100755 airflow/www/__init__.py mode change 100644 => 100755 airflow/www/api/__init__.py mode change 100644 => 100755 airflow/www/api/experimental/__init__.py mode change 100644 => 100755 airflow/www/api/experimental/endpoints.py mode change 100644 => 100755 airflow/www/app.py mode change 100644 => 100755 airflow/www/blueprints.py mode change 100644 => 100755 airflow/www/decorators.py mode change 100644 => 100755 airflow/www/extensions/__init__.py mode change 100644 => 100755 airflow/www/extensions/init_appbuilder.py mode change 100644 => 100755 airflow/www/extensions/init_appbuilder_links.py mode change 100644 => 100755 airflow/www/extensions/init_dagbag.py mode change 100644 => 100755 airflow/www/extensions/init_jinja_globals.py mode change 100644 => 100755 airflow/www/extensions/init_manifest_files.py mode change 100644 => 100755 airflow/www/extensions/init_security.py mode change 100644 => 100755 airflow/www/extensions/init_session.py mode change 100644 => 100755 airflow/www/extensions/init_views.py mode change 100644 => 100755 airflow/www/extensions/init_wsgi_middlewares.py mode change 100644 => 100755 airflow/www/forms.py mode change 100644 => 100755 airflow/www/gunicorn_config.py mode change 100644 => 100755 airflow/www/package.json mode change 100644 => 100755 airflow/www/security.py mode change 100644 => 100755 airflow/www/static/airflow.gif mode change 100644 => 100755 airflow/www/static/css/bootstrap-theme.css mode change 100644 => 100755 airflow/www/static/css/flash.css mode change 100644 => 100755 airflow/www/static/css/gantt.css mode change 100644 => 100755 airflow/www/static/css/graph.css mode change 100644 => 100755 airflow/www/static/css/main.css mode change 100644 => 100755 airflow/www/static/css/tree.css mode change 100644 => 100755 airflow/www/static/js/base.js mode change 100644 => 100755 airflow/www/static/js/connection_form.js mode change 100644 => 100755 airflow/www/static/js/datetime-utils.js mode change 100644 => 100755 airflow/www/static/js/gantt-chart-d3v2.js mode change 100644 => 100755 airflow/www/static/js/graph.js mode change 100644 => 100755 airflow/www/static/js/ie.js mode change 100644 => 100755 airflow/www/static/js/task-instances.js mode change 100644 => 100755 airflow/www/static/loading.gif mode change 100644 => 100755 airflow/www/static/pin.svg mode change 100644 => 100755 airflow/www/static/pin_100.png mode change 100644 => 100755 airflow/www/static/pin_25.png mode change 100644 => 100755 airflow/www/static/pin_32.png mode change 100644 => 100755 airflow/www/static/pin_35.png mode change 100644 => 100755 airflow/www/static/pin_40.png mode change 100644 => 100755 airflow/www/static/pin_large.png mode change 100644 => 100755 airflow/www/static/screenshots/gantt.png mode change 100644 => 100755 airflow/www/static/screenshots/graph.png mode change 100644 => 100755 airflow/www/static/screenshots/tree.png mode change 100644 => 100755 airflow/www/static/sort_asc.png mode change 100644 => 100755 airflow/www/static/sort_both.png mode change 100644 => 100755 airflow/www/static/sort_desc.png mode change 100644 => 100755 airflow/www/templates/airflow/chart.html mode change 100644 => 100755 airflow/www/templates/airflow/circles.html mode change 100644 => 100755 airflow/www/templates/airflow/code.html mode change 100644 => 100755 airflow/www/templates/airflow/config.html mode change 100644 => 100755 airflow/www/templates/airflow/confirm.html mode change 100644 => 100755 airflow/www/templates/airflow/conn_create.html mode change 100644 => 100755 airflow/www/templates/airflow/conn_edit.html mode change 100644 => 100755 airflow/www/templates/airflow/dag.html mode change 100644 => 100755 airflow/www/templates/airflow/dag_code.html mode change 100644 => 100755 airflow/www/templates/airflow/dag_details.html mode change 100644 => 100755 airflow/www/templates/airflow/dags.html mode change 100644 => 100755 airflow/www/templates/airflow/duration_chart.html mode change 100644 => 100755 airflow/www/templates/airflow/gantt.html mode change 100644 => 100755 airflow/www/templates/airflow/graph.html mode change 100644 => 100755 airflow/www/templates/airflow/master.html mode change 100644 => 100755 airflow/www/templates/airflow/model_list.html mode change 100644 => 100755 airflow/www/templates/airflow/noaccess.html mode change 100644 => 100755 airflow/www/templates/airflow/redoc.html mode change 100644 => 100755 airflow/www/templates/airflow/task.html mode change 100644 => 100755 airflow/www/templates/airflow/task_instance.html mode change 100644 => 100755 airflow/www/templates/airflow/ti_code.html mode change 100644 => 100755 airflow/www/templates/airflow/ti_log.html mode change 100644 => 100755 airflow/www/templates/airflow/traceback.html mode change 100644 => 100755 airflow/www/templates/airflow/tree.html mode change 100644 => 100755 airflow/www/templates/airflow/trigger.html mode change 100644 => 100755 airflow/www/templates/airflow/variable_edit.html mode change 100644 => 100755 airflow/www/templates/airflow/variable_list.html mode change 100644 => 100755 airflow/www/templates/airflow/version.html mode change 100644 => 100755 airflow/www/templates/airflow/xcom.html mode change 100644 => 100755 airflow/www/templates/analytics/google_analytics.html mode change 100644 => 100755 airflow/www/templates/analytics/metarouter.html mode change 100644 => 100755 airflow/www/templates/analytics/segment.html mode change 100644 => 100755 airflow/www/templates/appbuilder/flash.html mode change 100644 => 100755 airflow/www/templates/appbuilder/index.html mode change 100644 => 100755 airflow/www/templates/appbuilder/navbar.html mode change 100644 => 100755 airflow/www/templates/appbuilder/navbar_menu.html mode change 100644 => 100755 airflow/www/templates/appbuilder/navbar_right.html mode change 100644 => 100755 airflow/www/utils.py mode change 100644 => 100755 airflow/www/validators.py mode change 100644 => 100755 airflow/www/views.py mode change 100644 => 100755 airflow/www/webpack.config.js mode change 100644 => 100755 airflow/www/widgets.py mode change 100644 => 100755 airflow/www/yarn.lock mode change 100644 => 100755 breeze-complete mode change 100644 => 100755 codecov.yml mode change 100644 => 100755 pylintrc mode change 100644 => 100755 pytest.ini mode change 100644 => 100755 scripts/ci/docker-compose/local.yml mode change 100644 => 100755 setup.cfg mode change 100644 => 100755 setup.py mode change 100644 => 100755 yamllint-config.yml diff --git a/BREEZE.rst b/BREEZE.rst old mode 100644 new mode 100755 diff --git a/CHANGELOG.txt b/CHANGELOG.txt old mode 100644 new mode 100755 diff --git a/CI.rst b/CI.rst old mode 100644 new mode 100755 diff --git a/CODE_OF_CONDUCT.md b/CODE_OF_CONDUCT.md old mode 100644 new mode 100755 diff --git a/CONTRIBUTING.rst b/CONTRIBUTING.rst old mode 100644 new mode 100755 diff --git a/Dockerfile b/Dockerfile old mode 100644 new mode 100755 diff --git a/Dockerfile.ci b/Dockerfile.ci old mode 100644 new mode 100755 diff --git a/IMAGES.rst b/IMAGES.rst old mode 100644 new mode 100755 diff --git a/INSTALL b/INSTALL old mode 100644 new mode 100755 diff --git a/LICENSE b/LICENSE old mode 100644 new mode 100755 diff --git a/LOCAL_VIRTUALENV.rst b/LOCAL_VIRTUALENV.rst old mode 100644 new mode 100755 diff --git a/MANIFEST.in b/MANIFEST.in old mode 100644 new mode 100755 diff --git a/NOTICE b/NOTICE old mode 100644 new mode 100755 diff --git a/README.md b/README.md old mode 100644 new mode 100755 diff --git a/STATIC_CODE_CHECKS.rst b/STATIC_CODE_CHECKS.rst old mode 100644 new mode 100755 diff --git a/TESTING.rst b/TESTING.rst old mode 100644 new mode 100755 diff --git a/UPDATING.md b/UPDATING.md old mode 100644 new mode 100755 diff --git a/airflow/__init__.py b/airflow/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/__main__.py b/airflow/__main__.py old mode 100644 new mode 100755 diff --git a/airflow/alembic.ini b/airflow/alembic.ini old mode 100644 new mode 100755 diff --git a/airflow/api/__init__.py b/airflow/api/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/api/auth/__init__.py b/airflow/api/auth/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/api/auth/backend/__init__.py b/airflow/api/auth/backend/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/api/auth/backend/basic_auth.py b/airflow/api/auth/backend/basic_auth.py old mode 100644 new mode 100755 diff --git a/airflow/api/auth/backend/default.py b/airflow/api/auth/backend/default.py old mode 100644 new mode 100755 diff --git a/airflow/api/auth/backend/deny_all.py b/airflow/api/auth/backend/deny_all.py old mode 100644 new mode 100755 diff --git a/airflow/api/auth/backend/kerberos_auth.py b/airflow/api/auth/backend/kerberos_auth.py old mode 100644 new mode 100755 diff --git a/airflow/api/client/__init__.py b/airflow/api/client/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/api/client/api_client.py b/airflow/api/client/api_client.py old mode 100644 new mode 100755 diff --git a/airflow/api/client/json_client.py b/airflow/api/client/json_client.py old mode 100644 new mode 100755 diff --git a/airflow/api/client/local_client.py b/airflow/api/client/local_client.py old mode 100644 new mode 100755 diff --git a/airflow/api/common/__init__.py b/airflow/api/common/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/api/common/experimental/__init__.py b/airflow/api/common/experimental/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/api/common/experimental/delete_dag.py b/airflow/api/common/experimental/delete_dag.py old mode 100644 new mode 100755 diff --git a/airflow/api/common/experimental/get_code.py b/airflow/api/common/experimental/get_code.py old mode 100644 new mode 100755 diff --git a/airflow/api/common/experimental/get_dag_run_state.py b/airflow/api/common/experimental/get_dag_run_state.py old mode 100644 new mode 100755 diff --git a/airflow/api/common/experimental/get_dag_runs.py b/airflow/api/common/experimental/get_dag_runs.py old mode 100644 new mode 100755 diff --git a/airflow/api/common/experimental/get_lineage.py b/airflow/api/common/experimental/get_lineage.py old mode 100644 new mode 100755 diff --git a/airflow/api/common/experimental/get_task.py b/airflow/api/common/experimental/get_task.py old mode 100644 new mode 100755 diff --git a/airflow/api/common/experimental/get_task_instance.py b/airflow/api/common/experimental/get_task_instance.py old mode 100644 new mode 100755 diff --git a/airflow/api/common/experimental/mark_tasks.py b/airflow/api/common/experimental/mark_tasks.py old mode 100644 new mode 100755 diff --git a/airflow/api/common/experimental/pool.py b/airflow/api/common/experimental/pool.py old mode 100644 new mode 100755 diff --git a/airflow/api/common/experimental/trigger_dag.py b/airflow/api/common/experimental/trigger_dag.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/__init__.py b/airflow/api_connexion/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/__init__.py b/airflow/api_connexion/endpoints/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/config_endpoint.py b/airflow/api_connexion/endpoints/config_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/connection_endpoint.py b/airflow/api_connexion/endpoints/connection_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/dag_endpoint.py b/airflow/api_connexion/endpoints/dag_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/dag_run_endpoint.py b/airflow/api_connexion/endpoints/dag_run_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/dag_source_endpoint.py b/airflow/api_connexion/endpoints/dag_source_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/event_log_endpoint.py b/airflow/api_connexion/endpoints/event_log_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/extra_link_endpoint.py b/airflow/api_connexion/endpoints/extra_link_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/health_endpoint.py b/airflow/api_connexion/endpoints/health_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/import_error_endpoint.py b/airflow/api_connexion/endpoints/import_error_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/log_endpoint.py b/airflow/api_connexion/endpoints/log_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/pool_endpoint.py b/airflow/api_connexion/endpoints/pool_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/task_endpoint.py b/airflow/api_connexion/endpoints/task_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/task_instance_endpoint.py b/airflow/api_connexion/endpoints/task_instance_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/variable_endpoint.py b/airflow/api_connexion/endpoints/variable_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/version_endpoint.py b/airflow/api_connexion/endpoints/version_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/endpoints/xcom_endpoint.py b/airflow/api_connexion/endpoints/xcom_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/exceptions.py b/airflow/api_connexion/exceptions.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/openapi/v1.yaml b/airflow/api_connexion/openapi/v1.yaml old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/parameters.py b/airflow/api_connexion/parameters.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/__init__.py b/airflow/api_connexion/schemas/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/common_schema.py b/airflow/api_connexion/schemas/common_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/config_schema.py b/airflow/api_connexion/schemas/config_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/connection_schema.py b/airflow/api_connexion/schemas/connection_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/dag_run_schema.py b/airflow/api_connexion/schemas/dag_run_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/dag_schema.py b/airflow/api_connexion/schemas/dag_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/dag_source_schema.py b/airflow/api_connexion/schemas/dag_source_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/enum_schemas.py b/airflow/api_connexion/schemas/enum_schemas.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/error_schema.py b/airflow/api_connexion/schemas/error_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/event_log_schema.py b/airflow/api_connexion/schemas/event_log_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/health_schema.py b/airflow/api_connexion/schemas/health_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/log_schema.py b/airflow/api_connexion/schemas/log_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/pool_schema.py b/airflow/api_connexion/schemas/pool_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/task_schema.py b/airflow/api_connexion/schemas/task_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/variable_schema.py b/airflow/api_connexion/schemas/variable_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/version_schema.py b/airflow/api_connexion/schemas/version_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/schemas/xcom_schema.py b/airflow/api_connexion/schemas/xcom_schema.py old mode 100644 new mode 100755 diff --git a/airflow/api_connexion/security.py b/airflow/api_connexion/security.py old mode 100644 new mode 100755 diff --git a/airflow/cli/__init__.py b/airflow/cli/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/cli/cli_parser.py b/airflow/cli/cli_parser.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/__init__.py b/airflow/cli/commands/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/celery_command.py b/airflow/cli/commands/celery_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/config_command.py b/airflow/cli/commands/config_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/connection_command.py b/airflow/cli/commands/connection_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/dag_command.py b/airflow/cli/commands/dag_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/db_command.py b/airflow/cli/commands/db_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/info_command.py b/airflow/cli/commands/info_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/kerberos_command.py b/airflow/cli/commands/kerberos_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/legacy_commands.py b/airflow/cli/commands/legacy_commands.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/plugins_command.py b/airflow/cli/commands/plugins_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/pool_command.py b/airflow/cli/commands/pool_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/role_command.py b/airflow/cli/commands/role_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/rotate_fernet_key_command.py b/airflow/cli/commands/rotate_fernet_key_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/scheduler_command.py b/airflow/cli/commands/scheduler_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/sync_perm_command.py b/airflow/cli/commands/sync_perm_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/user_command.py b/airflow/cli/commands/user_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/variable_command.py b/airflow/cli/commands/variable_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/version_command.py b/airflow/cli/commands/version_command.py old mode 100644 new mode 100755 diff --git a/airflow/cli/commands/webserver_command.py b/airflow/cli/commands/webserver_command.py old mode 100644 new mode 100755 diff --git a/airflow/config_templates/__init__.py b/airflow/config_templates/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/config_templates/airflow_local_settings.py b/airflow/config_templates/airflow_local_settings.py old mode 100644 new mode 100755 diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml old mode 100644 new mode 100755 diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg old mode 100644 new mode 100755 diff --git a/airflow/config_templates/default_celery.py b/airflow/config_templates/default_celery.py old mode 100644 new mode 100755 diff --git a/airflow/config_templates/default_test.cfg b/airflow/config_templates/default_test.cfg old mode 100644 new mode 100755 diff --git a/airflow/config_templates/default_webserver_config.py b/airflow/config_templates/default_webserver_config.py old mode 100644 new mode 100755 diff --git a/airflow/configuration.py b/airflow/configuration.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/__init__.py b/airflow/contrib/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/__init__.py b/airflow/contrib/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/aws_athena_hook.py b/airflow/contrib/hooks/aws_athena_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/aws_datasync_hook.py b/airflow/contrib/hooks/aws_datasync_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/aws_dynamodb_hook.py b/airflow/contrib/hooks/aws_dynamodb_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/aws_firehose_hook.py b/airflow/contrib/hooks/aws_firehose_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/aws_glue_catalog_hook.py b/airflow/contrib/hooks/aws_glue_catalog_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/aws_hook.py b/airflow/contrib/hooks/aws_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/aws_lambda_hook.py b/airflow/contrib/hooks/aws_lambda_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/aws_logs_hook.py b/airflow/contrib/hooks/aws_logs_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/aws_sns_hook.py b/airflow/contrib/hooks/aws_sns_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/aws_sqs_hook.py b/airflow/contrib/hooks/aws_sqs_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/azure_container_instance_hook.py b/airflow/contrib/hooks/azure_container_instance_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/azure_container_registry_hook.py b/airflow/contrib/hooks/azure_container_registry_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/azure_container_volume_hook.py b/airflow/contrib/hooks/azure_container_volume_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/azure_cosmos_hook.py b/airflow/contrib/hooks/azure_cosmos_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/azure_data_lake_hook.py b/airflow/contrib/hooks/azure_data_lake_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/azure_fileshare_hook.py b/airflow/contrib/hooks/azure_fileshare_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/cassandra_hook.py b/airflow/contrib/hooks/cassandra_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/cloudant_hook.py b/airflow/contrib/hooks/cloudant_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/databricks_hook.py b/airflow/contrib/hooks/databricks_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/datadog_hook.py b/airflow/contrib/hooks/datadog_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/datastore_hook.py b/airflow/contrib/hooks/datastore_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/dingding_hook.py b/airflow/contrib/hooks/dingding_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/discord_webhook_hook.py b/airflow/contrib/hooks/discord_webhook_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/emr_hook.py b/airflow/contrib/hooks/emr_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/fs_hook.py b/airflow/contrib/hooks/fs_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/ftp_hook.py b/airflow/contrib/hooks/ftp_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_api_base_hook.py b/airflow/contrib/hooks/gcp_api_base_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_bigtable_hook.py b/airflow/contrib/hooks/gcp_bigtable_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_cloud_build_hook.py b/airflow/contrib/hooks/gcp_cloud_build_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_compute_hook.py b/airflow/contrib/hooks/gcp_compute_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_container_hook.py b/airflow/contrib/hooks/gcp_container_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_dataflow_hook.py b/airflow/contrib/hooks/gcp_dataflow_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_dataproc_hook.py b/airflow/contrib/hooks/gcp_dataproc_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_dlp_hook.py b/airflow/contrib/hooks/gcp_dlp_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_function_hook.py b/airflow/contrib/hooks/gcp_function_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_kms_hook.py b/airflow/contrib/hooks/gcp_kms_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_mlengine_hook.py b/airflow/contrib/hooks/gcp_mlengine_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_natural_language_hook.py b/airflow/contrib/hooks/gcp_natural_language_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_pubsub_hook.py b/airflow/contrib/hooks/gcp_pubsub_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_spanner_hook.py b/airflow/contrib/hooks/gcp_spanner_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_speech_to_text_hook.py b/airflow/contrib/hooks/gcp_speech_to_text_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_sql_hook.py b/airflow/contrib/hooks/gcp_sql_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_tasks_hook.py b/airflow/contrib/hooks/gcp_tasks_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_text_to_speech_hook.py b/airflow/contrib/hooks/gcp_text_to_speech_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_transfer_hook.py b/airflow/contrib/hooks/gcp_transfer_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_translate_hook.py b/airflow/contrib/hooks/gcp_translate_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_video_intelligence_hook.py b/airflow/contrib/hooks/gcp_video_intelligence_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcp_vision_hook.py b/airflow/contrib/hooks/gcp_vision_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gcs_hook.py b/airflow/contrib/hooks/gcs_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/gdrive_hook.py b/airflow/contrib/hooks/gdrive_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/grpc_hook.py b/airflow/contrib/hooks/grpc_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/imap_hook.py b/airflow/contrib/hooks/imap_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/jenkins_hook.py b/airflow/contrib/hooks/jenkins_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/jira_hook.py b/airflow/contrib/hooks/jira_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/mongo_hook.py b/airflow/contrib/hooks/mongo_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/openfaas_hook.py b/airflow/contrib/hooks/openfaas_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/opsgenie_alert_hook.py b/airflow/contrib/hooks/opsgenie_alert_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/pagerduty_hook.py b/airflow/contrib/hooks/pagerduty_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/pinot_hook.py b/airflow/contrib/hooks/pinot_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/qubole_check_hook.py b/airflow/contrib/hooks/qubole_check_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/qubole_hook.py b/airflow/contrib/hooks/qubole_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/redis_hook.py b/airflow/contrib/hooks/redis_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/redshift_hook.py b/airflow/contrib/hooks/redshift_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/sagemaker_hook.py b/airflow/contrib/hooks/sagemaker_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/salesforce_hook.py b/airflow/contrib/hooks/salesforce_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/segment_hook.py b/airflow/contrib/hooks/segment_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/sftp_hook.py b/airflow/contrib/hooks/sftp_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/slack_webhook_hook.py b/airflow/contrib/hooks/slack_webhook_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/snowflake_hook.py b/airflow/contrib/hooks/snowflake_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/spark_jdbc_hook.py b/airflow/contrib/hooks/spark_jdbc_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/spark_sql_hook.py b/airflow/contrib/hooks/spark_sql_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/spark_submit_hook.py b/airflow/contrib/hooks/spark_submit_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/sqoop_hook.py b/airflow/contrib/hooks/sqoop_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/ssh_hook.py b/airflow/contrib/hooks/ssh_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/vertica_hook.py b/airflow/contrib/hooks/vertica_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/wasb_hook.py b/airflow/contrib/hooks/wasb_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/hooks/winrm_hook.py b/airflow/contrib/hooks/winrm_hook.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/__init__.py b/airflow/contrib/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/adls_list_operator.py b/airflow/contrib/operators/adls_list_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/adls_to_gcs.py b/airflow/contrib/operators/adls_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/aws_athena_operator.py b/airflow/contrib/operators/aws_athena_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/aws_sqs_publish_operator.py b/airflow/contrib/operators/aws_sqs_publish_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/awsbatch_operator.py b/airflow/contrib/operators/awsbatch_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/azure_container_instances_operator.py b/airflow/contrib/operators/azure_container_instances_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/azure_cosmos_operator.py b/airflow/contrib/operators/azure_cosmos_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/bigquery_check_operator.py b/airflow/contrib/operators/bigquery_check_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/bigquery_get_data.py b/airflow/contrib/operators/bigquery_get_data.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/bigquery_operator.py b/airflow/contrib/operators/bigquery_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/bigquery_table_delete_operator.py b/airflow/contrib/operators/bigquery_table_delete_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/bigquery_to_bigquery.py b/airflow/contrib/operators/bigquery_to_bigquery.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/bigquery_to_gcs.py b/airflow/contrib/operators/bigquery_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/bigquery_to_mysql_operator.py b/airflow/contrib/operators/bigquery_to_mysql_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/cassandra_to_gcs.py b/airflow/contrib/operators/cassandra_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/databricks_operator.py b/airflow/contrib/operators/databricks_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/dataflow_operator.py b/airflow/contrib/operators/dataflow_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/dataproc_operator.py b/airflow/contrib/operators/dataproc_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/datastore_export_operator.py b/airflow/contrib/operators/datastore_export_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/datastore_import_operator.py b/airflow/contrib/operators/datastore_import_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/dingding_operator.py b/airflow/contrib/operators/dingding_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/discord_webhook_operator.py b/airflow/contrib/operators/discord_webhook_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/docker_swarm_operator.py b/airflow/contrib/operators/docker_swarm_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/druid_operator.py b/airflow/contrib/operators/druid_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/dynamodb_to_s3.py b/airflow/contrib/operators/dynamodb_to_s3.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/ecs_operator.py b/airflow/contrib/operators/ecs_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/emr_add_steps_operator.py b/airflow/contrib/operators/emr_add_steps_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/emr_create_job_flow_operator.py b/airflow/contrib/operators/emr_create_job_flow_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/emr_terminate_job_flow_operator.py b/airflow/contrib/operators/emr_terminate_job_flow_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/file_to_gcs.py b/airflow/contrib/operators/file_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/file_to_wasb.py b/airflow/contrib/operators/file_to_wasb.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_bigtable_operator.py b/airflow/contrib/operators/gcp_bigtable_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_cloud_build_operator.py b/airflow/contrib/operators/gcp_cloud_build_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_compute_operator.py b/airflow/contrib/operators/gcp_compute_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_container_operator.py b/airflow/contrib/operators/gcp_container_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_dlp_operator.py b/airflow/contrib/operators/gcp_dlp_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_function_operator.py b/airflow/contrib/operators/gcp_function_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_natural_language_operator.py b/airflow/contrib/operators/gcp_natural_language_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_spanner_operator.py b/airflow/contrib/operators/gcp_spanner_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_speech_to_text_operator.py b/airflow/contrib/operators/gcp_speech_to_text_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_sql_operator.py b/airflow/contrib/operators/gcp_sql_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_tasks_operator.py b/airflow/contrib/operators/gcp_tasks_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_text_to_speech_operator.py b/airflow/contrib/operators/gcp_text_to_speech_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_transfer_operator.py b/airflow/contrib/operators/gcp_transfer_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_translate_operator.py b/airflow/contrib/operators/gcp_translate_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_translate_speech_operator.py b/airflow/contrib/operators/gcp_translate_speech_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_video_intelligence_operator.py b/airflow/contrib/operators/gcp_video_intelligence_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcp_vision_operator.py b/airflow/contrib/operators/gcp_vision_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcs_acl_operator.py b/airflow/contrib/operators/gcs_acl_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcs_delete_operator.py b/airflow/contrib/operators/gcs_delete_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcs_download_operator.py b/airflow/contrib/operators/gcs_download_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcs_list_operator.py b/airflow/contrib/operators/gcs_list_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcs_operator.py b/airflow/contrib/operators/gcs_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcs_to_bq.py b/airflow/contrib/operators/gcs_to_bq.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcs_to_gcs.py b/airflow/contrib/operators/gcs_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcs_to_gcs_transfer_operator.py b/airflow/contrib/operators/gcs_to_gcs_transfer_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcs_to_gdrive_operator.py b/airflow/contrib/operators/gcs_to_gdrive_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/gcs_to_s3.py b/airflow/contrib/operators/gcs_to_s3.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/grpc_operator.py b/airflow/contrib/operators/grpc_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/hive_to_dynamodb.py b/airflow/contrib/operators/hive_to_dynamodb.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/imap_attachment_to_s3_operator.py b/airflow/contrib/operators/imap_attachment_to_s3_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/jenkins_job_trigger_operator.py b/airflow/contrib/operators/jenkins_job_trigger_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/jira_operator.py b/airflow/contrib/operators/jira_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/kubernetes_pod_operator.py b/airflow/contrib/operators/kubernetes_pod_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/mlengine_operator.py b/airflow/contrib/operators/mlengine_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/mongo_to_s3.py b/airflow/contrib/operators/mongo_to_s3.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/mssql_to_gcs.py b/airflow/contrib/operators/mssql_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/mysql_to_gcs.py b/airflow/contrib/operators/mysql_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/opsgenie_alert_operator.py b/airflow/contrib/operators/opsgenie_alert_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py b/airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/oracle_to_oracle_transfer.py b/airflow/contrib/operators/oracle_to_oracle_transfer.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/postgres_to_gcs_operator.py b/airflow/contrib/operators/postgres_to_gcs_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/pubsub_operator.py b/airflow/contrib/operators/pubsub_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/qubole_check_operator.py b/airflow/contrib/operators/qubole_check_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/qubole_operator.py b/airflow/contrib/operators/qubole_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/redis_publish_operator.py b/airflow/contrib/operators/redis_publish_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/s3_copy_object_operator.py b/airflow/contrib/operators/s3_copy_object_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/s3_delete_objects_operator.py b/airflow/contrib/operators/s3_delete_objects_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/s3_list_operator.py b/airflow/contrib/operators/s3_list_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/s3_to_gcs_operator.py b/airflow/contrib/operators/s3_to_gcs_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/s3_to_gcs_transfer_operator.py b/airflow/contrib/operators/s3_to_gcs_transfer_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/s3_to_sftp_operator.py b/airflow/contrib/operators/s3_to_sftp_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/sagemaker_base_operator.py b/airflow/contrib/operators/sagemaker_base_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/sagemaker_endpoint_config_operator.py b/airflow/contrib/operators/sagemaker_endpoint_config_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/sagemaker_endpoint_operator.py b/airflow/contrib/operators/sagemaker_endpoint_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/sagemaker_model_operator.py b/airflow/contrib/operators/sagemaker_model_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/sagemaker_training_operator.py b/airflow/contrib/operators/sagemaker_training_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/sagemaker_transform_operator.py b/airflow/contrib/operators/sagemaker_transform_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/sagemaker_tuning_operator.py b/airflow/contrib/operators/sagemaker_tuning_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/segment_track_event_operator.py b/airflow/contrib/operators/segment_track_event_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/sftp_operator.py b/airflow/contrib/operators/sftp_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/sftp_to_s3_operator.py b/airflow/contrib/operators/sftp_to_s3_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/slack_webhook_operator.py b/airflow/contrib/operators/slack_webhook_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/snowflake_operator.py b/airflow/contrib/operators/snowflake_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/sns_publish_operator.py b/airflow/contrib/operators/sns_publish_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/spark_jdbc_operator.py b/airflow/contrib/operators/spark_jdbc_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/spark_sql_operator.py b/airflow/contrib/operators/spark_sql_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/spark_submit_operator.py b/airflow/contrib/operators/spark_submit_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/sql_to_gcs.py b/airflow/contrib/operators/sql_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/sqoop_operator.py b/airflow/contrib/operators/sqoop_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/ssh_operator.py b/airflow/contrib/operators/ssh_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/vertica_operator.py b/airflow/contrib/operators/vertica_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/vertica_to_hive.py b/airflow/contrib/operators/vertica_to_hive.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/vertica_to_mysql.py b/airflow/contrib/operators/vertica_to_mysql.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/wasb_delete_blob_operator.py b/airflow/contrib/operators/wasb_delete_blob_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/operators/winrm_operator.py b/airflow/contrib/operators/winrm_operator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/secrets/__init__.py b/airflow/contrib/secrets/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/secrets/aws_secrets_manager.py b/airflow/contrib/secrets/aws_secrets_manager.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/secrets/aws_systems_manager.py b/airflow/contrib/secrets/aws_systems_manager.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/secrets/gcp_secrets_manager.py b/airflow/contrib/secrets/gcp_secrets_manager.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/secrets/hashicorp_vault.py b/airflow/contrib/secrets/hashicorp_vault.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/__init__.py b/airflow/contrib/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/aws_athena_sensor.py b/airflow/contrib/sensors/aws_athena_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/aws_glue_catalog_partition_sensor.py b/airflow/contrib/sensors/aws_glue_catalog_partition_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/aws_redshift_cluster_sensor.py b/airflow/contrib/sensors/aws_redshift_cluster_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/aws_sqs_sensor.py b/airflow/contrib/sensors/aws_sqs_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/azure_cosmos_sensor.py b/airflow/contrib/sensors/azure_cosmos_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/bash_sensor.py b/airflow/contrib/sensors/bash_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/bigquery_sensor.py b/airflow/contrib/sensors/bigquery_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/cassandra_record_sensor.py b/airflow/contrib/sensors/cassandra_record_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/cassandra_table_sensor.py b/airflow/contrib/sensors/cassandra_table_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/celery_queue_sensor.py b/airflow/contrib/sensors/celery_queue_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/datadog_sensor.py b/airflow/contrib/sensors/datadog_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/emr_base_sensor.py b/airflow/contrib/sensors/emr_base_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/emr_job_flow_sensor.py b/airflow/contrib/sensors/emr_job_flow_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/emr_step_sensor.py b/airflow/contrib/sensors/emr_step_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/file_sensor.py b/airflow/contrib/sensors/file_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/ftp_sensor.py b/airflow/contrib/sensors/ftp_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/gcp_transfer_sensor.py b/airflow/contrib/sensors/gcp_transfer_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/gcs_sensor.py b/airflow/contrib/sensors/gcs_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/hdfs_sensor.py b/airflow/contrib/sensors/hdfs_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/imap_attachment_sensor.py b/airflow/contrib/sensors/imap_attachment_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/jira_sensor.py b/airflow/contrib/sensors/jira_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/mongo_sensor.py b/airflow/contrib/sensors/mongo_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/pubsub_sensor.py b/airflow/contrib/sensors/pubsub_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/python_sensor.py b/airflow/contrib/sensors/python_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/qubole_sensor.py b/airflow/contrib/sensors/qubole_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/redis_key_sensor.py b/airflow/contrib/sensors/redis_key_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/redis_pub_sub_sensor.py b/airflow/contrib/sensors/redis_pub_sub_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/sagemaker_base_sensor.py b/airflow/contrib/sensors/sagemaker_base_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/sagemaker_endpoint_sensor.py b/airflow/contrib/sensors/sagemaker_endpoint_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/sagemaker_training_sensor.py b/airflow/contrib/sensors/sagemaker_training_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/sagemaker_transform_sensor.py b/airflow/contrib/sensors/sagemaker_transform_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/sagemaker_tuning_sensor.py b/airflow/contrib/sensors/sagemaker_tuning_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/sftp_sensor.py b/airflow/contrib/sensors/sftp_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/wasb_sensor.py b/airflow/contrib/sensors/wasb_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/sensors/weekday_sensor.py b/airflow/contrib/sensors/weekday_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/task_runner/__init__.py b/airflow/contrib/task_runner/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/task_runner/cgroup_task_runner.py b/airflow/contrib/task_runner/cgroup_task_runner.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/utils/__init__.py b/airflow/contrib/utils/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/utils/gcp_field_sanitizer.py b/airflow/contrib/utils/gcp_field_sanitizer.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/utils/gcp_field_validator.py b/airflow/contrib/utils/gcp_field_validator.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/utils/log/__init__.py b/airflow/contrib/utils/log/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/utils/log/task_handler_with_custom_formatter.py b/airflow/contrib/utils/log/task_handler_with_custom_formatter.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/utils/mlengine_operator_utils.py b/airflow/contrib/utils/mlengine_operator_utils.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/utils/mlengine_prediction_summary.py b/airflow/contrib/utils/mlengine_prediction_summary.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/utils/sendgrid.py b/airflow/contrib/utils/sendgrid.py old mode 100644 new mode 100755 diff --git a/airflow/contrib/utils/weekday.py b/airflow/contrib/utils/weekday.py old mode 100644 new mode 100755 diff --git a/airflow/dag/__init__.py b/airflow/dag/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/dag/base_dag.py b/airflow/dag/base_dag.py old mode 100644 new mode 100755 diff --git a/airflow/decorators.py b/airflow/decorators.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/__init__.py b/airflow/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_bash_operator.py b/airflow/example_dags/example_bash_operator.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_branch_operator.py b/airflow/example_dags/example_branch_operator.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_branch_python_dop_operator_3.py b/airflow/example_dags/example_branch_python_dop_operator_3.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_complex.py b/airflow/example_dags/example_complex.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_external_task_marker_dag.py b/airflow/example_dags/example_external_task_marker_dag.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_kubernetes_executor.py b/airflow/example_dags/example_kubernetes_executor.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_kubernetes_executor_config.py b/airflow/example_dags/example_kubernetes_executor_config.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_latest_only.py b/airflow/example_dags/example_latest_only.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_latest_only_with_trigger.py b/airflow/example_dags/example_latest_only_with_trigger.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_nested_branch_dag.py b/airflow/example_dags/example_nested_branch_dag.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_passing_params_via_test_command.py b/airflow/example_dags/example_passing_params_via_test_command.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_python_operator.py b/airflow/example_dags/example_python_operator.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_short_circuit_operator.py b/airflow/example_dags/example_short_circuit_operator.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_skip_dag.py b/airflow/example_dags/example_skip_dag.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_subdag_operator.py b/airflow/example_dags/example_subdag_operator.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_trigger_controller_dag.py b/airflow/example_dags/example_trigger_controller_dag.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_trigger_target_dag.py b/airflow/example_dags/example_trigger_target_dag.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_xcom.py b/airflow/example_dags/example_xcom.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/example_xcomargs.py b/airflow/example_dags/example_xcomargs.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/input_notebook.ipynb b/airflow/example_dags/input_notebook.ipynb old mode 100644 new mode 100755 diff --git a/airflow/example_dags/libs/__init__.py b/airflow/example_dags/libs/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/libs/helper.py b/airflow/example_dags/libs/helper.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/subdags/__init__.py b/airflow/example_dags/subdags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/subdags/subdag.py b/airflow/example_dags/subdags/subdag.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/test_utils.py b/airflow/example_dags/test_utils.py old mode 100644 new mode 100755 diff --git a/airflow/example_dags/tutorial.py b/airflow/example_dags/tutorial.py old mode 100644 new mode 100755 diff --git a/airflow/exceptions.py b/airflow/exceptions.py old mode 100644 new mode 100755 diff --git a/airflow/executors/__init__.py b/airflow/executors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py old mode 100644 new mode 100755 diff --git a/airflow/executors/celery_executor.py b/airflow/executors/celery_executor.py old mode 100644 new mode 100755 diff --git a/airflow/executors/dask_executor.py b/airflow/executors/dask_executor.py old mode 100644 new mode 100755 diff --git a/airflow/executors/debug_executor.py b/airflow/executors/debug_executor.py old mode 100644 new mode 100755 diff --git a/airflow/executors/executor_loader.py b/airflow/executors/executor_loader.py old mode 100644 new mode 100755 diff --git a/airflow/executors/kubernetes_executor.py b/airflow/executors/kubernetes_executor.py old mode 100644 new mode 100755 diff --git a/airflow/executors/local_executor.py b/airflow/executors/local_executor.py old mode 100644 new mode 100755 diff --git a/airflow/executors/sequential_executor.py b/airflow/executors/sequential_executor.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/S3_hook.py b/airflow/hooks/S3_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/__init__.py b/airflow/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/base_hook.py b/airflow/hooks/base_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/dbapi_hook.py b/airflow/hooks/dbapi_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/docker_hook.py b/airflow/hooks/docker_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/druid_hook.py b/airflow/hooks/druid_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/filesystem.py b/airflow/hooks/filesystem.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/hdfs_hook.py b/airflow/hooks/hdfs_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/hive_hooks.py b/airflow/hooks/hive_hooks.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/http_hook.py b/airflow/hooks/http_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/jdbc_hook.py b/airflow/hooks/jdbc_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/mssql_hook.py b/airflow/hooks/mssql_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/mysql_hook.py b/airflow/hooks/mysql_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/oracle_hook.py b/airflow/hooks/oracle_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/pig_hook.py b/airflow/hooks/pig_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/postgres_hook.py b/airflow/hooks/postgres_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/presto_hook.py b/airflow/hooks/presto_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/samba_hook.py b/airflow/hooks/samba_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/slack_hook.py b/airflow/hooks/slack_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/sqlite_hook.py b/airflow/hooks/sqlite_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/webhdfs_hook.py b/airflow/hooks/webhdfs_hook.py old mode 100644 new mode 100755 diff --git a/airflow/hooks/zendesk_hook.py b/airflow/hooks/zendesk_hook.py old mode 100644 new mode 100755 diff --git a/airflow/jobs/__init__.py b/airflow/jobs/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/jobs/backfill_job.py b/airflow/jobs/backfill_job.py old mode 100644 new mode 100755 diff --git a/airflow/jobs/base_job.py b/airflow/jobs/base_job.py old mode 100644 new mode 100755 diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py old mode 100644 new mode 100755 diff --git a/airflow/jobs/scheduler_job.py b/airflow/jobs/scheduler_job.py old mode 100644 new mode 100755 diff --git a/airflow/kubernetes/__init__.py b/airflow/kubernetes/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/kubernetes/k8s_model.py b/airflow/kubernetes/k8s_model.py old mode 100644 new mode 100755 diff --git a/airflow/kubernetes/kube_client.py b/airflow/kubernetes/kube_client.py old mode 100644 new mode 100755 diff --git a/airflow/kubernetes/pod.py b/airflow/kubernetes/pod.py old mode 100644 new mode 100755 diff --git a/airflow/kubernetes/pod_generator.py b/airflow/kubernetes/pod_generator.py old mode 100644 new mode 100755 diff --git a/airflow/kubernetes/pod_launcher.py b/airflow/kubernetes/pod_launcher.py old mode 100644 new mode 100755 diff --git a/airflow/kubernetes/pod_runtime_info_env.py b/airflow/kubernetes/pod_runtime_info_env.py old mode 100644 new mode 100755 diff --git a/airflow/kubernetes/refresh_config.py b/airflow/kubernetes/refresh_config.py old mode 100644 new mode 100755 diff --git a/airflow/kubernetes/secret.py b/airflow/kubernetes/secret.py old mode 100644 new mode 100755 diff --git a/airflow/kubernetes/volume.py b/airflow/kubernetes/volume.py old mode 100644 new mode 100755 diff --git a/airflow/kubernetes/volume_mount.py b/airflow/kubernetes/volume_mount.py old mode 100644 new mode 100755 diff --git a/airflow/kubernetes/worker_configuration.py b/airflow/kubernetes/worker_configuration.py old mode 100644 new mode 100755 diff --git a/airflow/lineage/__init__.py b/airflow/lineage/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/lineage/entities.py b/airflow/lineage/entities.py old mode 100644 new mode 100755 diff --git a/airflow/logging_config.py b/airflow/logging_config.py old mode 100644 new mode 100755 diff --git a/airflow/macros/__init__.py b/airflow/macros/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/macros/hive.py b/airflow/macros/hive.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/__init__.py b/airflow/migrations/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/env.py b/airflow/migrations/env.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/script.py.mako b/airflow/migrations/script.py.mako old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/004c1210f153_increase_queue_name_size_limit.py b/airflow/migrations/versions/004c1210f153_increase_queue_name_size_limit.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/03bc53e68815_add_sm_dag_index.py b/airflow/migrations/versions/03bc53e68815_add_sm_dag_index.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/05f30312d566_merge_heads.py b/airflow/migrations/versions/05f30312d566_merge_heads.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/0a2a5b66e19d_add_task_reschedule_table.py b/airflow/migrations/versions/0a2a5b66e19d_add_task_reschedule_table.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/0e2a74e0fc9f_add_time_zone_awareness.py b/airflow/migrations/versions/0e2a74e0fc9f_add_time_zone_awareness.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/127d2bf2dfa7_add_dag_id_state_index_on_dag_run_table.py b/airflow/migrations/versions/127d2bf2dfa7_add_dag_id_state_index_on_dag_run_table.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/13eb55f81627_for_compatibility.py b/airflow/migrations/versions/13eb55f81627_for_compatibility.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/1507a7289a2f_create_is_encrypted.py b/airflow/migrations/versions/1507a7289a2f_create_is_encrypted.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/1968acfc09e3_add_is_encrypted_column_to_variable_.py b/airflow/migrations/versions/1968acfc09e3_add_is_encrypted_column_to_variable_.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/1b38cef5b76e_add_dagrun.py b/airflow/migrations/versions/1b38cef5b76e_add_dagrun.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/211e584da130_add_ti_state_index.py b/airflow/migrations/versions/211e584da130_add_ti_state_index.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/27c6a30d7c24_add_executor_config_to_task_instance.py b/airflow/migrations/versions/27c6a30d7c24_add_executor_config_to_task_instance.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/2e541a1dcfed_task_duration.py b/airflow/migrations/versions/2e541a1dcfed_task_duration.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/2e82aab8ef20_rename_user_table.py b/airflow/migrations/versions/2e82aab8ef20_rename_user_table.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/338e90f54d61_more_logging_into_task_isntance.py b/airflow/migrations/versions/338e90f54d61_more_logging_into_task_isntance.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py b/airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/3c20cacc0044_add_dagrun_run_type.py b/airflow/migrations/versions/3c20cacc0044_add_dagrun_run_type.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/40e67319e3a9_dagrun_config.py b/airflow/migrations/versions/40e67319e3a9_dagrun_config.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/41f5f12752f8_add_superuser_field.py b/airflow/migrations/versions/41f5f12752f8_add_superuser_field.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/4446e08588_dagrun_start_end.py b/airflow/migrations/versions/4446e08588_dagrun_start_end.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py b/airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/502898887f84_adding_extra_to_log.py b/airflow/migrations/versions/502898887f84_adding_extra_to_log.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/52d714495f0_job_id_indices.py b/airflow/migrations/versions/52d714495f0_job_id_indices.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/561833c1c74b_add_password_column_to_user.py b/airflow/migrations/versions/561833c1c74b_add_password_column_to_user.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/5e7d17757c7a_add_pid_field_to_taskinstance.py b/airflow/migrations/versions/5e7d17757c7a_add_pid_field_to_taskinstance.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/64de9cddf6c9_add_task_fails_journal_table.py b/airflow/migrations/versions/64de9cddf6c9_add_task_fails_journal_table.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/6e96a59344a4_make_taskinstance_pool_not_nullable.py b/airflow/migrations/versions/6e96a59344a4_make_taskinstance_pool_not_nullable.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/74effc47d867_change_datetime_to_datetime2_6_on_mssql_.py b/airflow/migrations/versions/74effc47d867_change_datetime_to_datetime2_6_on_mssql_.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/7939bcff74ba_add_dagtags_table.py b/airflow/migrations/versions/7939bcff74ba_add_dagtags_table.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/8504051e801b_xcom_dag_task_indices.py b/airflow/migrations/versions/8504051e801b_xcom_dag_task_indices.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/852ae6c715af_add_rendered_task_instance_fields_table.py b/airflow/migrations/versions/852ae6c715af_add_rendered_task_instance_fields_table.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/856955da8476_fix_sqlite_foreign_key.py b/airflow/migrations/versions/856955da8476_fix_sqlite_foreign_key.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py b/airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/8d48763f6d53_add_unique_constraint_to_conn_id.py b/airflow/migrations/versions/8d48763f6d53_add_unique_constraint_to_conn_id.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/8f966b9c467a_set_conn_type_as_non_nullable.py b/airflow/migrations/versions/8f966b9c467a_set_conn_type_as_non_nullable.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/939bb1e647c8_task_reschedule_fk_on_cascade_delete.py b/airflow/migrations/versions/939bb1e647c8_task_reschedule_fk_on_cascade_delete.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/947454bf1dff_add_ti_job_id_index.py b/airflow/migrations/versions/947454bf1dff_add_ti_job_id_index.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/952da73b5eff_add_dag_code_table.py b/airflow/migrations/versions/952da73b5eff_add_dag_code_table.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/9635ae0956e7_index_faskfail.py b/airflow/migrations/versions/9635ae0956e7_index_faskfail.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/__init__.py b/airflow/migrations/versions/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/a4c2fd67d16b_add_pool_slots_field_to_task_instance.py b/airflow/migrations/versions/a4c2fd67d16b_add_pool_slots_field_to_task_instance.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/a56c9515abdc_remove_dag_stat_table.py b/airflow/migrations/versions/a56c9515abdc_remove_dag_stat_table.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/a66efa278eea_add_precision_to_execution_date_in_mysql.py b/airflow/migrations/versions/a66efa278eea_add_precision_to_execution_date_in_mysql.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/b0125267960b_merge_heads.py b/airflow/migrations/versions/b0125267960b_merge_heads.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/b25a55525161_increase_length_of_pool_name.py b/airflow/migrations/versions/b25a55525161_increase_length_of_pool_name.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/b3b105409875_add_root_dag_id_to_dag.py b/airflow/migrations/versions/b3b105409875_add_root_dag_id_to_dag.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/bba5a7cfc896_add_a_column_to_track_the_encryption_.py b/airflow/migrations/versions/bba5a7cfc896_add_a_column_to_track_the_encryption_.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/bbc73705a13e_add_notification_sent_column_to_sla_miss.py b/airflow/migrations/versions/bbc73705a13e_add_notification_sent_column_to_sla_miss.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/bbf4a7ad0465_remove_id_column_from_xcom.py b/airflow/migrations/versions/bbf4a7ad0465_remove_id_column_from_xcom.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/bdaa763e6c56_make_xcom_value_column_a_large_binary.py b/airflow/migrations/versions/bdaa763e6c56_make_xcom_value_column_a_large_binary.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/bf00311e1990_add_index_to_taskinstance.py b/airflow/migrations/versions/bf00311e1990_add_index_to_taskinstance.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/c8ffec048a3b_add_fields_to_dag.py b/airflow/migrations/versions/c8ffec048a3b_add_fields_to_dag.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py b/airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/cf5dc11e79ad_drop_user_and_chart.py b/airflow/migrations/versions/cf5dc11e79ad_drop_user_and_chart.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/d2ae31099d61_increase_text_size_for_mysql.py b/airflow/migrations/versions/d2ae31099d61_increase_text_size_for_mysql.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/d38e04c12aa2_add_serialized_dag_table.py b/airflow/migrations/versions/d38e04c12aa2_add_serialized_dag_table.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/da3f683c3a5a_add_dag_hash_column_to_serialized_dag_.py b/airflow/migrations/versions/da3f683c3a5a_add_dag_hash_column_to_serialized_dag_.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/dd25f486b8ea_add_idx_log_dag.py b/airflow/migrations/versions/dd25f486b8ea_add_idx_log_dag.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/dd4ecb8fbee3_add_schedule_interval_to_dag.py b/airflow/migrations/versions/dd4ecb8fbee3_add_schedule_interval_to_dag.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/e3a246e0dc1_current_schema.py b/airflow/migrations/versions/e3a246e0dc1_current_schema.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/f23433877c24_fix_mysql_not_null_constraint.py b/airflow/migrations/versions/f23433877c24_fix_mysql_not_null_constraint.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/f2ca10b85618_add_dag_stats_table.py b/airflow/migrations/versions/f2ca10b85618_add_dag_stats_table.py old mode 100644 new mode 100755 diff --git a/airflow/migrations/versions/fe461863935f_increase_length_for_connection_password.py b/airflow/migrations/versions/fe461863935f_increase_length_for_connection_password.py old mode 100644 new mode 100755 diff --git a/airflow/models/__init__.py b/airflow/models/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/models/base.py b/airflow/models/base.py old mode 100644 new mode 100755 diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py old mode 100644 new mode 100755 diff --git a/airflow/models/connection.py b/airflow/models/connection.py old mode 100644 new mode 100755 diff --git a/airflow/models/crypto.py b/airflow/models/crypto.py old mode 100644 new mode 100755 diff --git a/airflow/models/dag.py b/airflow/models/dag.py old mode 100644 new mode 100755 diff --git a/airflow/models/dagbag.py b/airflow/models/dagbag.py old mode 100644 new mode 100755 diff --git a/airflow/models/dagcode.py b/airflow/models/dagcode.py old mode 100644 new mode 100755 diff --git a/airflow/models/dagpickle.py b/airflow/models/dagpickle.py old mode 100644 new mode 100755 diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py old mode 100644 new mode 100755 diff --git a/airflow/models/errors.py b/airflow/models/errors.py old mode 100644 new mode 100755 diff --git a/airflow/models/kubernetes.py b/airflow/models/kubernetes.py old mode 100644 new mode 100755 diff --git a/airflow/models/log.py b/airflow/models/log.py old mode 100644 new mode 100755 diff --git a/airflow/models/pool.py b/airflow/models/pool.py old mode 100644 new mode 100755 diff --git a/airflow/models/renderedtifields.py b/airflow/models/renderedtifields.py old mode 100644 new mode 100755 diff --git a/airflow/models/serialized_dag.py b/airflow/models/serialized_dag.py old mode 100644 new mode 100755 diff --git a/airflow/models/skipmixin.py b/airflow/models/skipmixin.py old mode 100644 new mode 100755 diff --git a/airflow/models/slamiss.py b/airflow/models/slamiss.py old mode 100644 new mode 100755 diff --git a/airflow/models/taskfail.py b/airflow/models/taskfail.py old mode 100644 new mode 100755 diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py old mode 100644 new mode 100755 diff --git a/airflow/models/taskreschedule.py b/airflow/models/taskreschedule.py old mode 100644 new mode 100755 diff --git a/airflow/models/variable.py b/airflow/models/variable.py old mode 100644 new mode 100755 diff --git a/airflow/models/xcom.py b/airflow/models/xcom.py old mode 100644 new mode 100755 diff --git a/airflow/models/xcom_arg.py b/airflow/models/xcom_arg.py old mode 100644 new mode 100755 diff --git a/airflow/mypy/__init__.py b/airflow/mypy/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/mypy/plugin/__init__.py b/airflow/mypy/plugin/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/mypy/plugin/decorators.py b/airflow/mypy/plugin/decorators.py old mode 100644 new mode 100755 diff --git a/airflow/operators/__init__.py b/airflow/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/operators/bash.py b/airflow/operators/bash.py old mode 100644 new mode 100755 diff --git a/airflow/operators/bash_operator.py b/airflow/operators/bash_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/branch_operator.py b/airflow/operators/branch_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/check_operator.py b/airflow/operators/check_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/dagrun_operator.py b/airflow/operators/dagrun_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/docker_operator.py b/airflow/operators/docker_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/druid_check_operator.py b/airflow/operators/druid_check_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/dummy_operator.py b/airflow/operators/dummy_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/email.py b/airflow/operators/email.py old mode 100644 new mode 100755 diff --git a/airflow/operators/email_operator.py b/airflow/operators/email_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/gcs_to_s3.py b/airflow/operators/gcs_to_s3.py old mode 100644 new mode 100755 diff --git a/airflow/operators/generic_transfer.py b/airflow/operators/generic_transfer.py old mode 100644 new mode 100755 diff --git a/airflow/operators/google_api_to_s3_transfer.py b/airflow/operators/google_api_to_s3_transfer.py old mode 100644 new mode 100755 diff --git a/airflow/operators/hive_operator.py b/airflow/operators/hive_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/hive_stats_operator.py b/airflow/operators/hive_stats_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/hive_to_druid.py b/airflow/operators/hive_to_druid.py old mode 100644 new mode 100755 diff --git a/airflow/operators/hive_to_mysql.py b/airflow/operators/hive_to_mysql.py old mode 100644 new mode 100755 diff --git a/airflow/operators/hive_to_samba_operator.py b/airflow/operators/hive_to_samba_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/http_operator.py b/airflow/operators/http_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/jdbc_operator.py b/airflow/operators/jdbc_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/latest_only_operator.py b/airflow/operators/latest_only_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/mssql_operator.py b/airflow/operators/mssql_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/mssql_to_hive.py b/airflow/operators/mssql_to_hive.py old mode 100644 new mode 100755 diff --git a/airflow/operators/mysql_operator.py b/airflow/operators/mysql_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/mysql_to_hive.py b/airflow/operators/mysql_to_hive.py old mode 100644 new mode 100755 diff --git a/airflow/operators/oracle_operator.py b/airflow/operators/oracle_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/papermill_operator.py b/airflow/operators/papermill_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/pig_operator.py b/airflow/operators/pig_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/postgres_operator.py b/airflow/operators/postgres_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/presto_check_operator.py b/airflow/operators/presto_check_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/presto_to_mysql.py b/airflow/operators/presto_to_mysql.py old mode 100644 new mode 100755 diff --git a/airflow/operators/python.py b/airflow/operators/python.py old mode 100644 new mode 100755 diff --git a/airflow/operators/python_operator.py b/airflow/operators/python_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/redshift_to_s3_operator.py b/airflow/operators/redshift_to_s3_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/s3_file_transform_operator.py b/airflow/operators/s3_file_transform_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/s3_to_hive_operator.py b/airflow/operators/s3_to_hive_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/s3_to_redshift_operator.py b/airflow/operators/s3_to_redshift_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/slack_operator.py b/airflow/operators/slack_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/sql.py b/airflow/operators/sql.py old mode 100644 new mode 100755 diff --git a/airflow/operators/sql_branch_operator.py b/airflow/operators/sql_branch_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/sqlite_operator.py b/airflow/operators/sqlite_operator.py old mode 100644 new mode 100755 diff --git a/airflow/operators/subdag_operator.py b/airflow/operators/subdag_operator.py old mode 100644 new mode 100755 diff --git a/airflow/plugins_manager.py b/airflow/plugins_manager.py old mode 100644 new mode 100755 diff --git a/airflow/providers/__init__.py b/airflow/providers/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/amazon/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/README.md b/airflow/providers/amazon/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/__init__.py b/airflow/providers/amazon/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/__init__.py b/airflow/providers/amazon/aws/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/example_dags/__init__.py b/airflow/providers/amazon/aws/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/example_dags/example_datasync_1.py b/airflow/providers/amazon/aws/example_dags/example_datasync_1.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/example_dags/example_datasync_2.py b/airflow/providers/amazon/aws/example_dags/example_datasync_2.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/example_dags/example_ecs_fargate.py b/airflow/providers/amazon/aws/example_dags/example_ecs_fargate.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/example_dags/example_emr_job_flow_automatic_steps.py b/airflow/providers/amazon/aws/example_dags/example_emr_job_flow_automatic_steps.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/example_dags/example_emr_job_flow_manual_steps.py b/airflow/providers/amazon/aws/example_dags/example_emr_job_flow_manual_steps.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/example_dags/example_google_api_to_s3_transfer_advanced.py b/airflow/providers/amazon/aws/example_dags/example_google_api_to_s3_transfer_advanced.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/example_dags/example_google_api_to_s3_transfer_basic.py b/airflow/providers/amazon/aws/example_dags/example_google_api_to_s3_transfer_basic.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/example_dags/example_s3_bucket.py b/airflow/providers/amazon/aws/example_dags/example_s3_bucket.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py b/airflow/providers/amazon/aws/example_dags/example_s3_to_redshift.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/__init__.py b/airflow/providers/amazon/aws/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/athena.py b/airflow/providers/amazon/aws/hooks/athena.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/aws_dynamodb.py b/airflow/providers/amazon/aws/hooks/aws_dynamodb.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/base_aws.py b/airflow/providers/amazon/aws/hooks/base_aws.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/batch_client.py b/airflow/providers/amazon/aws/hooks/batch_client.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/batch_waiters.json b/airflow/providers/amazon/aws/hooks/batch_waiters.json old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/batch_waiters.py b/airflow/providers/amazon/aws/hooks/batch_waiters.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/cloud_formation.py b/airflow/providers/amazon/aws/hooks/cloud_formation.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/datasync.py b/airflow/providers/amazon/aws/hooks/datasync.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/ec2.py b/airflow/providers/amazon/aws/hooks/ec2.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/emr.py b/airflow/providers/amazon/aws/hooks/emr.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/glue.py b/airflow/providers/amazon/aws/hooks/glue.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/glue_catalog.py b/airflow/providers/amazon/aws/hooks/glue_catalog.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/kinesis.py b/airflow/providers/amazon/aws/hooks/kinesis.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/lambda_function.py b/airflow/providers/amazon/aws/hooks/lambda_function.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/logs.py b/airflow/providers/amazon/aws/hooks/logs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/redshift.py b/airflow/providers/amazon/aws/hooks/redshift.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/s3.py b/airflow/providers/amazon/aws/hooks/s3.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/sagemaker.py b/airflow/providers/amazon/aws/hooks/sagemaker.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/ses.py b/airflow/providers/amazon/aws/hooks/ses.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/sns.py b/airflow/providers/amazon/aws/hooks/sns.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/sqs.py b/airflow/providers/amazon/aws/hooks/sqs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/hooks/step_function.py b/airflow/providers/amazon/aws/hooks/step_function.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/log/__init__.py b/airflow/providers/amazon/aws/log/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/log/s3_task_handler.py b/airflow/providers/amazon/aws/log/s3_task_handler.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/__init__.py b/airflow/providers/amazon/aws/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/athena.py b/airflow/providers/amazon/aws/operators/athena.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/batch.py b/airflow/providers/amazon/aws/operators/batch.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/cloud_formation.py b/airflow/providers/amazon/aws/operators/cloud_formation.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/datasync.py b/airflow/providers/amazon/aws/operators/datasync.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/ec2_start_instance.py b/airflow/providers/amazon/aws/operators/ec2_start_instance.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/ec2_stop_instance.py b/airflow/providers/amazon/aws/operators/ec2_stop_instance.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/ecs.py b/airflow/providers/amazon/aws/operators/ecs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/emr_add_steps.py b/airflow/providers/amazon/aws/operators/emr_add_steps.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/emr_create_job_flow.py b/airflow/providers/amazon/aws/operators/emr_create_job_flow.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/emr_modify_cluster.py b/airflow/providers/amazon/aws/operators/emr_modify_cluster.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/emr_terminate_job_flow.py b/airflow/providers/amazon/aws/operators/emr_terminate_job_flow.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/glue.py b/airflow/providers/amazon/aws/operators/glue.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/s3_bucket.py b/airflow/providers/amazon/aws/operators/s3_bucket.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/s3_copy_object.py b/airflow/providers/amazon/aws/operators/s3_copy_object.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/s3_delete_objects.py b/airflow/providers/amazon/aws/operators/s3_delete_objects.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/s3_file_transform.py b/airflow/providers/amazon/aws/operators/s3_file_transform.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/s3_list.py b/airflow/providers/amazon/aws/operators/s3_list.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/sagemaker_base.py b/airflow/providers/amazon/aws/operators/sagemaker_base.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/sagemaker_endpoint.py b/airflow/providers/amazon/aws/operators/sagemaker_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/sagemaker_endpoint_config.py b/airflow/providers/amazon/aws/operators/sagemaker_endpoint_config.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/sagemaker_model.py b/airflow/providers/amazon/aws/operators/sagemaker_model.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/sagemaker_processing.py b/airflow/providers/amazon/aws/operators/sagemaker_processing.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/sagemaker_training.py b/airflow/providers/amazon/aws/operators/sagemaker_training.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/sagemaker_transform.py b/airflow/providers/amazon/aws/operators/sagemaker_transform.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/sagemaker_tuning.py b/airflow/providers/amazon/aws/operators/sagemaker_tuning.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/sns.py b/airflow/providers/amazon/aws/operators/sns.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/sqs.py b/airflow/providers/amazon/aws/operators/sqs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/step_function_get_execution_output.py b/airflow/providers/amazon/aws/operators/step_function_get_execution_output.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/operators/step_function_start_execution.py b/airflow/providers/amazon/aws/operators/step_function_start_execution.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/secrets/__init__.py b/airflow/providers/amazon/aws/secrets/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/secrets/secrets_manager.py b/airflow/providers/amazon/aws/secrets/secrets_manager.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/secrets/systems_manager.py b/airflow/providers/amazon/aws/secrets/systems_manager.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/__init__.py b/airflow/providers/amazon/aws/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/athena.py b/airflow/providers/amazon/aws/sensors/athena.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/cloud_formation.py b/airflow/providers/amazon/aws/sensors/cloud_formation.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/ec2_instance_state.py b/airflow/providers/amazon/aws/sensors/ec2_instance_state.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/emr_base.py b/airflow/providers/amazon/aws/sensors/emr_base.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/emr_job_flow.py b/airflow/providers/amazon/aws/sensors/emr_job_flow.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/emr_step.py b/airflow/providers/amazon/aws/sensors/emr_step.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/glue.py b/airflow/providers/amazon/aws/sensors/glue.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/glue_catalog_partition.py b/airflow/providers/amazon/aws/sensors/glue_catalog_partition.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/redshift.py b/airflow/providers/amazon/aws/sensors/redshift.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/s3_key.py b/airflow/providers/amazon/aws/sensors/s3_key.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/s3_keys_unchanged.py b/airflow/providers/amazon/aws/sensors/s3_keys_unchanged.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/s3_prefix.py b/airflow/providers/amazon/aws/sensors/s3_prefix.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/sagemaker_base.py b/airflow/providers/amazon/aws/sensors/sagemaker_base.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/sagemaker_endpoint.py b/airflow/providers/amazon/aws/sensors/sagemaker_endpoint.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/sagemaker_training.py b/airflow/providers/amazon/aws/sensors/sagemaker_training.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/sagemaker_transform.py b/airflow/providers/amazon/aws/sensors/sagemaker_transform.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/sagemaker_tuning.py b/airflow/providers/amazon/aws/sensors/sagemaker_tuning.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/sqs.py b/airflow/providers/amazon/aws/sensors/sqs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/sensors/step_function_execution.py b/airflow/providers/amazon/aws/sensors/step_function_execution.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/transfers/__init__.py b/airflow/providers/amazon/aws/transfers/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/transfers/dynamodb_to_s3.py b/airflow/providers/amazon/aws/transfers/dynamodb_to_s3.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/transfers/gcs_to_s3.py b/airflow/providers/amazon/aws/transfers/gcs_to_s3.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/transfers/google_api_to_s3.py b/airflow/providers/amazon/aws/transfers/google_api_to_s3.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/transfers/hive_to_dynamodb.py b/airflow/providers/amazon/aws/transfers/hive_to_dynamodb.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/transfers/imap_attachment_to_s3.py b/airflow/providers/amazon/aws/transfers/imap_attachment_to_s3.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/transfers/mongo_to_s3.py b/airflow/providers/amazon/aws/transfers/mongo_to_s3.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/transfers/mysql_to_s3.py b/airflow/providers/amazon/aws/transfers/mysql_to_s3.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/transfers/redshift_to_s3.py b/airflow/providers/amazon/aws/transfers/redshift_to_s3.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/transfers/s3_to_redshift.py b/airflow/providers/amazon/aws/transfers/s3_to_redshift.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/transfers/s3_to_sftp.py b/airflow/providers/amazon/aws/transfers/s3_to_sftp.py old mode 100644 new mode 100755 diff --git a/airflow/providers/amazon/aws/transfers/sftp_to_s3.py b/airflow/providers/amazon/aws/transfers/sftp_to_s3.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/__init__.py b/airflow/providers/apache/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/cassandra/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/apache/cassandra/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/cassandra/README.md b/airflow/providers/apache/cassandra/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/cassandra/__init__.py b/airflow/providers/apache/cassandra/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/cassandra/example_dags/__init__.py b/airflow/providers/apache/cassandra/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/cassandra/example_dags/example_cassandra_dag.py b/airflow/providers/apache/cassandra/example_dags/example_cassandra_dag.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/cassandra/hooks/__init__.py b/airflow/providers/apache/cassandra/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/cassandra/hooks/cassandra.py b/airflow/providers/apache/cassandra/hooks/cassandra.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/cassandra/sensors/__init__.py b/airflow/providers/apache/cassandra/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/cassandra/sensors/record.py b/airflow/providers/apache/cassandra/sensors/record.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/cassandra/sensors/table.py b/airflow/providers/apache/cassandra/sensors/table.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/druid/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/apache/druid/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/druid/README.md b/airflow/providers/apache/druid/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/druid/__init__.py b/airflow/providers/apache/druid/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/druid/hooks/__init__.py b/airflow/providers/apache/druid/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/druid/hooks/druid.py b/airflow/providers/apache/druid/hooks/druid.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/druid/operators/__init__.py b/airflow/providers/apache/druid/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/druid/operators/druid.py b/airflow/providers/apache/druid/operators/druid.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/druid/operators/druid_check.py b/airflow/providers/apache/druid/operators/druid_check.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/druid/transfers/__init__.py b/airflow/providers/apache/druid/transfers/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/druid/transfers/hive_to_druid.py b/airflow/providers/apache/druid/transfers/hive_to_druid.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hdfs/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/apache/hdfs/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hdfs/README.md b/airflow/providers/apache/hdfs/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hdfs/__init__.py b/airflow/providers/apache/hdfs/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hdfs/hooks/__init__.py b/airflow/providers/apache/hdfs/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hdfs/hooks/hdfs.py b/airflow/providers/apache/hdfs/hooks/hdfs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hdfs/hooks/webhdfs.py b/airflow/providers/apache/hdfs/hooks/webhdfs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hdfs/sensors/__init__.py b/airflow/providers/apache/hdfs/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hdfs/sensors/hdfs.py b/airflow/providers/apache/hdfs/sensors/hdfs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hdfs/sensors/web_hdfs.py b/airflow/providers/apache/hdfs/sensors/web_hdfs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/apache/hive/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/README.md b/airflow/providers/apache/hive/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/__init__.py b/airflow/providers/apache/hive/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/example_dags/__init__.py b/airflow/providers/apache/hive/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/example_dags/example_twitter_README.md b/airflow/providers/apache/hive/example_dags/example_twitter_README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/example_dags/example_twitter_dag.py b/airflow/providers/apache/hive/example_dags/example_twitter_dag.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/hooks/__init__.py b/airflow/providers/apache/hive/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/hooks/hive.py b/airflow/providers/apache/hive/hooks/hive.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/operators/__init__.py b/airflow/providers/apache/hive/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/operators/hive.py b/airflow/providers/apache/hive/operators/hive.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/operators/hive_stats.py b/airflow/providers/apache/hive/operators/hive_stats.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/sensors/__init__.py b/airflow/providers/apache/hive/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/sensors/hive_partition.py b/airflow/providers/apache/hive/sensors/hive_partition.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/sensors/metastore_partition.py b/airflow/providers/apache/hive/sensors/metastore_partition.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/sensors/named_hive_partition.py b/airflow/providers/apache/hive/sensors/named_hive_partition.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/transfers/__init__.py b/airflow/providers/apache/hive/transfers/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/transfers/hive_to_mysql.py b/airflow/providers/apache/hive/transfers/hive_to_mysql.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/transfers/hive_to_samba.py b/airflow/providers/apache/hive/transfers/hive_to_samba.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/transfers/mssql_to_hive.py b/airflow/providers/apache/hive/transfers/mssql_to_hive.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/transfers/mysql_to_hive.py b/airflow/providers/apache/hive/transfers/mysql_to_hive.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/transfers/s3_to_hive.py b/airflow/providers/apache/hive/transfers/s3_to_hive.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/hive/transfers/vertica_to_hive.py b/airflow/providers/apache/hive/transfers/vertica_to_hive.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kafka/example_dags/__init__.py b/airflow/providers/apache/kafka/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py b/airflow/providers/apache/kafka/example_dags/example_kafka_dag.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kafka/hooks/__init__.py b/airflow/providers/apache/kafka/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_consumer_hook.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py b/airflow/providers/apache/kafka/hooks/kafka_producer_hook.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kafka/sensors/__init__.py b/airflow/providers/apache/kafka/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kafka/sensors/kafka_sensor.py b/airflow/providers/apache/kafka/sensors/kafka_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kylin/__init__.py b/airflow/providers/apache/kylin/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kylin/example_dags/__init__.py b/airflow/providers/apache/kylin/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kylin/example_dags/example_kylin_dag.py b/airflow/providers/apache/kylin/example_dags/example_kylin_dag.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kylin/hooks/__init__.py b/airflow/providers/apache/kylin/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kylin/hooks/kylin.py b/airflow/providers/apache/kylin/hooks/kylin.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kylin/operators/__init__.py b/airflow/providers/apache/kylin/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/kylin/operators/kylin_cube.py b/airflow/providers/apache/kylin/operators/kylin_cube.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/livy/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/apache/livy/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/livy/README.md b/airflow/providers/apache/livy/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/livy/__init__.py b/airflow/providers/apache/livy/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/livy/example_dags/__init__.py b/airflow/providers/apache/livy/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/livy/example_dags/example_livy.py b/airflow/providers/apache/livy/example_dags/example_livy.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/livy/hooks/__init__.py b/airflow/providers/apache/livy/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/livy/hooks/livy.py b/airflow/providers/apache/livy/hooks/livy.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/livy/operators/__init__.py b/airflow/providers/apache/livy/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/livy/operators/livy.py b/airflow/providers/apache/livy/operators/livy.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/livy/sensors/__init__.py b/airflow/providers/apache/livy/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/livy/sensors/livy.py b/airflow/providers/apache/livy/sensors/livy.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/pig/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/apache/pig/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/pig/README.md b/airflow/providers/apache/pig/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/pig/__init__.py b/airflow/providers/apache/pig/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/pig/example_dags/__init__.py b/airflow/providers/apache/pig/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/pig/example_dags/example_pig.py b/airflow/providers/apache/pig/example_dags/example_pig.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/pig/hooks/__init__.py b/airflow/providers/apache/pig/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/pig/hooks/pig.py b/airflow/providers/apache/pig/hooks/pig.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/pig/operators/__init__.py b/airflow/providers/apache/pig/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/pig/operators/pig.py b/airflow/providers/apache/pig/operators/pig.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/pinot/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/apache/pinot/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/pinot/README.md b/airflow/providers/apache/pinot/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/pinot/__init__.py b/airflow/providers/apache/pinot/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/pinot/hooks/__init__.py b/airflow/providers/apache/pinot/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/pinot/hooks/pinot.py b/airflow/providers/apache/pinot/hooks/pinot.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/spark/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/apache/spark/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/spark/README.md b/airflow/providers/apache/spark/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/spark/__init__.py b/airflow/providers/apache/spark/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/spark/example_dags/__init__.py b/airflow/providers/apache/spark/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/spark/example_dags/example_spark_dag.py b/airflow/providers/apache/spark/example_dags/example_spark_dag.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/spark/hooks/__init__.py b/airflow/providers/apache/spark/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/spark/hooks/spark_jdbc.py b/airflow/providers/apache/spark/hooks/spark_jdbc.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/spark/hooks/spark_jdbc_script.py b/airflow/providers/apache/spark/hooks/spark_jdbc_script.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/spark/hooks/spark_sql.py b/airflow/providers/apache/spark/hooks/spark_sql.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/spark/hooks/spark_submit.py b/airflow/providers/apache/spark/hooks/spark_submit.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/spark/operators/__init__.py b/airflow/providers/apache/spark/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/spark/operators/spark_jdbc.py b/airflow/providers/apache/spark/operators/spark_jdbc.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/spark/operators/spark_sql.py b/airflow/providers/apache/spark/operators/spark_sql.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/spark/operators/spark_submit.py b/airflow/providers/apache/spark/operators/spark_submit.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/sqoop/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/apache/sqoop/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/sqoop/README.md b/airflow/providers/apache/sqoop/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/sqoop/__init__.py b/airflow/providers/apache/sqoop/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/sqoop/hooks/__init__.py b/airflow/providers/apache/sqoop/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/sqoop/hooks/sqoop.py b/airflow/providers/apache/sqoop/hooks/sqoop.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/sqoop/operators/__init__.py b/airflow/providers/apache/sqoop/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/apache/sqoop/operators/sqoop.py b/airflow/providers/apache/sqoop/operators/sqoop.py old mode 100644 new mode 100755 diff --git a/airflow/providers/celery/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/celery/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/celery/README.md b/airflow/providers/celery/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/celery/__init__.py b/airflow/providers/celery/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/celery/sensors/__init__.py b/airflow/providers/celery/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/celery/sensors/celery_queue.py b/airflow/providers/celery/sensors/celery_queue.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cloudant/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/cloudant/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/cloudant/README.md b/airflow/providers/cloudant/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/cloudant/__init__.py b/airflow/providers/cloudant/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cloudant/hooks/__init__.py b/airflow/providers/cloudant/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cloudant/hooks/cloudant.py b/airflow/providers/cloudant/hooks/cloudant.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cncf/__init__.py b/airflow/providers/cncf/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cncf/kubernetes/__init__.py b/airflow/providers/cncf/kubernetes/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cncf/kubernetes/example_dags/__init__.py b/airflow/providers/cncf/kubernetes/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cncf/kubernetes/example_dags/example_kubernetes.py b/airflow/providers/cncf/kubernetes/example_dags/example_kubernetes.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cncf/kubernetes/example_dags/example_spark_kubernetes.py b/airflow/providers/cncf/kubernetes/example_dags/example_spark_kubernetes.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cncf/kubernetes/example_dags/example_spark_kubernetes_spark_pi.yaml b/airflow/providers/cncf/kubernetes/example_dags/example_spark_kubernetes_spark_pi.yaml old mode 100644 new mode 100755 diff --git a/airflow/providers/cncf/kubernetes/hooks/__init__.py b/airflow/providers/cncf/kubernetes/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cncf/kubernetes/hooks/kubernetes.py b/airflow/providers/cncf/kubernetes/hooks/kubernetes.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cncf/kubernetes/operators/__init__.py b/airflow/providers/cncf/kubernetes/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cncf/kubernetes/operators/kubernetes_pod.py b/airflow/providers/cncf/kubernetes/operators/kubernetes_pod.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py b/airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cncf/kubernetes/sensors/__init__.py b/airflow/providers/cncf/kubernetes/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/cncf/kubernetes/sensors/spark_kubernetes.py b/airflow/providers/cncf/kubernetes/sensors/spark_kubernetes.py old mode 100644 new mode 100755 diff --git a/airflow/providers/databricks/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/databricks/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/databricks/README.md b/airflow/providers/databricks/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/databricks/__init__.py b/airflow/providers/databricks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/databricks/example_dags/__init__.py b/airflow/providers/databricks/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/databricks/example_dags/example_databricks.py b/airflow/providers/databricks/example_dags/example_databricks.py old mode 100644 new mode 100755 diff --git a/airflow/providers/databricks/hooks/__init__.py b/airflow/providers/databricks/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/databricks/hooks/databricks.py b/airflow/providers/databricks/hooks/databricks.py old mode 100644 new mode 100755 diff --git a/airflow/providers/databricks/operators/__init__.py b/airflow/providers/databricks/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/databricks/operators/databricks.py b/airflow/providers/databricks/operators/databricks.py old mode 100644 new mode 100755 diff --git a/airflow/providers/datadog/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/datadog/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/datadog/README.md b/airflow/providers/datadog/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/datadog/__init__.py b/airflow/providers/datadog/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/datadog/hooks/__init__.py b/airflow/providers/datadog/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/datadog/hooks/datadog.py b/airflow/providers/datadog/hooks/datadog.py old mode 100644 new mode 100755 diff --git a/airflow/providers/datadog/sensors/__init__.py b/airflow/providers/datadog/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/datadog/sensors/datadog.py b/airflow/providers/datadog/sensors/datadog.py old mode 100644 new mode 100755 diff --git a/airflow/providers/dependencies.json b/airflow/providers/dependencies.json old mode 100644 new mode 100755 diff --git a/airflow/providers/dingding/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/dingding/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/dingding/README.md b/airflow/providers/dingding/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/dingding/__init__.py b/airflow/providers/dingding/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/dingding/example_dags/__init__.py b/airflow/providers/dingding/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/dingding/example_dags/example_dingding.py b/airflow/providers/dingding/example_dags/example_dingding.py old mode 100644 new mode 100755 diff --git a/airflow/providers/dingding/hooks/__init__.py b/airflow/providers/dingding/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/dingding/hooks/dingding.py b/airflow/providers/dingding/hooks/dingding.py old mode 100644 new mode 100755 diff --git a/airflow/providers/dingding/operators/__init__.py b/airflow/providers/dingding/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/dingding/operators/dingding.py b/airflow/providers/dingding/operators/dingding.py old mode 100644 new mode 100755 diff --git a/airflow/providers/discord/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/discord/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/discord/README.md b/airflow/providers/discord/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/discord/__init__.py b/airflow/providers/discord/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/discord/hooks/__init__.py b/airflow/providers/discord/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/discord/hooks/discord_webhook.py b/airflow/providers/discord/hooks/discord_webhook.py old mode 100644 new mode 100755 diff --git a/airflow/providers/discord/operators/__init__.py b/airflow/providers/discord/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/discord/operators/discord_webhook.py b/airflow/providers/discord/operators/discord_webhook.py old mode 100644 new mode 100755 diff --git a/airflow/providers/docker/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/docker/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/docker/README.md b/airflow/providers/docker/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/docker/__init__.py b/airflow/providers/docker/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/docker/example_dags/__init__.py b/airflow/providers/docker/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/docker/example_dags/example_docker.py b/airflow/providers/docker/example_dags/example_docker.py old mode 100644 new mode 100755 diff --git a/airflow/providers/docker/example_dags/example_docker_copy_data.py b/airflow/providers/docker/example_dags/example_docker_copy_data.py old mode 100644 new mode 100755 diff --git a/airflow/providers/docker/example_dags/example_docker_swarm.py b/airflow/providers/docker/example_dags/example_docker_swarm.py old mode 100644 new mode 100755 diff --git a/airflow/providers/docker/hooks/__init__.py b/airflow/providers/docker/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/docker/hooks/docker.py b/airflow/providers/docker/hooks/docker.py old mode 100644 new mode 100755 diff --git a/airflow/providers/docker/operators/__init__.py b/airflow/providers/docker/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/docker/operators/docker.py b/airflow/providers/docker/operators/docker.py old mode 100644 new mode 100755 diff --git a/airflow/providers/docker/operators/docker_swarm.py b/airflow/providers/docker/operators/docker_swarm.py old mode 100644 new mode 100755 diff --git a/airflow/providers/elasticsearch/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/elasticsearch/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/elasticsearch/README.md b/airflow/providers/elasticsearch/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/elasticsearch/__init__.py b/airflow/providers/elasticsearch/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/elasticsearch/hooks/__init__.py b/airflow/providers/elasticsearch/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/elasticsearch/hooks/elasticsearch.py b/airflow/providers/elasticsearch/hooks/elasticsearch.py old mode 100644 new mode 100755 diff --git a/airflow/providers/elasticsearch/log/__init__.py b/airflow/providers/elasticsearch/log/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/elasticsearch/log/es_task_handler.py b/airflow/providers/elasticsearch/log/es_task_handler.py old mode 100644 new mode 100755 diff --git a/airflow/providers/exasol/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/exasol/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/exasol/README.md b/airflow/providers/exasol/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/exasol/__init__.py b/airflow/providers/exasol/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/exasol/hooks/__init__.py b/airflow/providers/exasol/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/exasol/hooks/exasol.py b/airflow/providers/exasol/hooks/exasol.py old mode 100644 new mode 100755 diff --git a/airflow/providers/exasol/operators/__init__.py b/airflow/providers/exasol/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/exasol/operators/exasol.py b/airflow/providers/exasol/operators/exasol.py old mode 100644 new mode 100755 diff --git a/airflow/providers/facebook/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/facebook/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/facebook/README.md b/airflow/providers/facebook/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/facebook/__init__.py b/airflow/providers/facebook/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/facebook/ads/__init__.py b/airflow/providers/facebook/ads/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/facebook/ads/hooks/__init__.py b/airflow/providers/facebook/ads/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/facebook/ads/hooks/ads.py b/airflow/providers/facebook/ads/hooks/ads.py old mode 100644 new mode 100755 diff --git a/airflow/providers/ftp/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/ftp/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/ftp/README.md b/airflow/providers/ftp/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/ftp/__init__.py b/airflow/providers/ftp/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/ftp/hooks/__init__.py b/airflow/providers/ftp/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/ftp/hooks/ftp.py b/airflow/providers/ftp/hooks/ftp.py old mode 100644 new mode 100755 diff --git a/airflow/providers/ftp/sensors/__init__.py b/airflow/providers/ftp/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/ftp/sensors/ftp.py b/airflow/providers/ftp/sensors/ftp.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/google/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/google/README.md b/airflow/providers/google/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/google/__init__.py b/airflow/providers/google/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/ads/__init__.py b/airflow/providers/google/ads/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/ads/example_dags/__init__.py b/airflow/providers/google/ads/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/ads/example_dags/example_ads.py b/airflow/providers/google/ads/example_dags/example_ads.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/ads/hooks/__init__.py b/airflow/providers/google/ads/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/ads/hooks/ads.py b/airflow/providers/google/ads/hooks/ads.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/ads/operators/__init__.py b/airflow/providers/google/ads/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/ads/operators/ads.py b/airflow/providers/google/ads/operators/ads.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/ads/transfers/__init__.py b/airflow/providers/google/ads/transfers/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/ads/transfers/ads_to_gcs.py b/airflow/providers/google/ads/transfers/ads_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/__init__.py b/airflow/providers/google/cloud/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/_internal_client/__init__.py b/airflow/providers/google/cloud/_internal_client/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/_internal_client/secret_manager_client.py b/airflow/providers/google/cloud/_internal_client/secret_manager_client.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/__init__.py b/airflow/providers/google/cloud/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_classification.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_extraction.py b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_extraction.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py b/airflow/providers/google/cloud/example_dags/example_automl_nl_text_sentiment.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_automl_tables.py b/airflow/providers/google/cloud/example_dags/example_automl_tables.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_automl_translation.py b/airflow/providers/google/cloud/example_dags/example_automl_translation.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py b/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_classification.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py b/airflow/providers/google/cloud/example_dags/example_automl_video_intelligence_tracking.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_automl_vision_classification.py b/airflow/providers/google/cloud/example_dags/example_automl_vision_classification.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py b/airflow/providers/google/cloud/example_dags/example_automl_vision_object_detection.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_bigquery_dts.py b/airflow/providers/google/cloud/example_dags/example_bigquery_dts.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_bigquery_operations.py b/airflow/providers/google/cloud/example_dags/example_bigquery_operations.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_bigquery_queries.py b/airflow/providers/google/cloud/example_dags/example_bigquery_queries.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_bigquery_query.sql b/airflow/providers/google/cloud/example_dags/example_bigquery_query.sql old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_bigquery_sensors.py b/airflow/providers/google/cloud/example_dags/example_bigquery_sensors.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_bigquery_to_bigquery.py b/airflow/providers/google/cloud/example_dags/example_bigquery_to_bigquery.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_bigquery_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_bigquery_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_bigquery_transfer.py b/airflow/providers/google/cloud/example_dags/example_bigquery_transfer.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_bigtable.py b/airflow/providers/google/cloud/example_dags/example_bigtable.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_build.py b/airflow/providers/google/cloud/example_dags/example_cloud_build.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_build.yaml b/airflow/providers/google/cloud/example_dags/example_cloud_build.yaml old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_memorystore.py b/airflow/providers/google/cloud/example_dags/example_cloud_memorystore.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_sql.py b/airflow/providers/google/cloud/example_dags/example_cloud_sql.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py b/airflow/providers/google/cloud/example_dags/example_cloud_sql_query.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_aws.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py b/airflow/providers/google/cloud/example_dags/example_cloud_storage_transfer_service_gcp.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_compute.py b/airflow/providers/google/cloud/example_dags/example_compute.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_compute_igm.py b/airflow/providers/google/cloud/example_dags/example_compute_igm.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_datacatalog.py b/airflow/providers/google/cloud/example_dags/example_datacatalog.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_dataflow.py b/airflow/providers/google/cloud/example_dags/example_dataflow.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_datafusion.py b/airflow/providers/google/cloud/example_dags/example_datafusion.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_dataprep.py b/airflow/providers/google/cloud/example_dags/example_dataprep.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_dataproc.py b/airflow/providers/google/cloud/example_dags/example_dataproc.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_datastore.py b/airflow/providers/google/cloud/example_dags/example_datastore.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_dlp.py b/airflow/providers/google/cloud/example_dags/example_dlp.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_facebook_ads_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_functions.py b/airflow/providers/google/cloud/example_dags/example_functions.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_gcs.py b/airflow/providers/google/cloud/example_dags/example_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_gcs_to_bigquery.py b/airflow/providers/google/cloud/example_dags/example_gcs_to_bigquery.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_gcs_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_gcs_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_gcs_to_sftp.py b/airflow/providers/google/cloud/example_dags/example_gcs_to_sftp.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_kubernetes_engine.py b/airflow/providers/google/cloud/example_dags/example_kubernetes_engine.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_life_sciences.py b/airflow/providers/google/cloud/example_dags/example_life_sciences.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_local_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_local_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_mlengine.py b/airflow/providers/google/cloud/example_dags/example_mlengine.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_natural_language.py b/airflow/providers/google/cloud/example_dags/example_natural_language.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_postgres_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_presto_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_pubsub.py b/airflow/providers/google/cloud/example_dags/example_pubsub.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_sftp_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_sheets_to_gcs.py b/airflow/providers/google/cloud/example_dags/example_sheets_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_spanner.py b/airflow/providers/google/cloud/example_dags/example_spanner.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_spanner.sql b/airflow/providers/google/cloud/example_dags/example_spanner.sql old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_speech_to_text.py b/airflow/providers/google/cloud/example_dags/example_speech_to_text.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_stackdriver.py b/airflow/providers/google/cloud/example_dags/example_stackdriver.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_tasks.py b/airflow/providers/google/cloud/example_dags/example_tasks.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_text_to_speech.py b/airflow/providers/google/cloud/example_dags/example_text_to_speech.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_translate.py b/airflow/providers/google/cloud/example_dags/example_translate.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_translate_speech.py b/airflow/providers/google/cloud/example_dags/example_translate_speech.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_video_intelligence.py b/airflow/providers/google/cloud/example_dags/example_video_intelligence.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/example_dags/example_vision.py b/airflow/providers/google/cloud/example_dags/example_vision.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/__init__.py b/airflow/providers/google/cloud/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/automl.py b/airflow/providers/google/cloud/hooks/automl.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/bigquery.py b/airflow/providers/google/cloud/hooks/bigquery.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/bigquery_dts.py b/airflow/providers/google/cloud/hooks/bigquery_dts.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/bigtable.py b/airflow/providers/google/cloud/hooks/bigtable.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/cloud_build.py b/airflow/providers/google/cloud/hooks/cloud_build.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/cloud_memorystore.py b/airflow/providers/google/cloud/hooks/cloud_memorystore.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/cloud_sql.py b/airflow/providers/google/cloud/hooks/cloud_sql.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py b/airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/compute.py b/airflow/providers/google/cloud/hooks/compute.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/datacatalog.py b/airflow/providers/google/cloud/hooks/datacatalog.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/dataflow.py b/airflow/providers/google/cloud/hooks/dataflow.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/datafusion.py b/airflow/providers/google/cloud/hooks/datafusion.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/dataprep.py b/airflow/providers/google/cloud/hooks/dataprep.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/dataproc.py b/airflow/providers/google/cloud/hooks/dataproc.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/datastore.py b/airflow/providers/google/cloud/hooks/datastore.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/dlp.py b/airflow/providers/google/cloud/hooks/dlp.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/functions.py b/airflow/providers/google/cloud/hooks/functions.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/gcs.py b/airflow/providers/google/cloud/hooks/gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/gdm.py b/airflow/providers/google/cloud/hooks/gdm.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/kms.py b/airflow/providers/google/cloud/hooks/kms.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/kubernetes_engine.py b/airflow/providers/google/cloud/hooks/kubernetes_engine.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/life_sciences.py b/airflow/providers/google/cloud/hooks/life_sciences.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/mlengine.py b/airflow/providers/google/cloud/hooks/mlengine.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/natural_language.py b/airflow/providers/google/cloud/hooks/natural_language.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/pubsub.py b/airflow/providers/google/cloud/hooks/pubsub.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/secret_manager.py b/airflow/providers/google/cloud/hooks/secret_manager.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/spanner.py b/airflow/providers/google/cloud/hooks/spanner.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/speech_to_text.py b/airflow/providers/google/cloud/hooks/speech_to_text.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/stackdriver.py b/airflow/providers/google/cloud/hooks/stackdriver.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/tasks.py b/airflow/providers/google/cloud/hooks/tasks.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/text_to_speech.py b/airflow/providers/google/cloud/hooks/text_to_speech.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/translate.py b/airflow/providers/google/cloud/hooks/translate.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/video_intelligence.py b/airflow/providers/google/cloud/hooks/video_intelligence.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/hooks/vision.py b/airflow/providers/google/cloud/hooks/vision.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/log/__init__.py b/airflow/providers/google/cloud/log/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/log/gcs_task_handler.py b/airflow/providers/google/cloud/log/gcs_task_handler.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/log/stackdriver_task_handler.py b/airflow/providers/google/cloud/log/stackdriver_task_handler.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/__init__.py b/airflow/providers/google/cloud/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/automl.py b/airflow/providers/google/cloud/operators/automl.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/bigquery.py b/airflow/providers/google/cloud/operators/bigquery.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/bigquery_dts.py b/airflow/providers/google/cloud/operators/bigquery_dts.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/bigtable.py b/airflow/providers/google/cloud/operators/bigtable.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/cloud_build.py b/airflow/providers/google/cloud/operators/cloud_build.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/cloud_memorystore.py b/airflow/providers/google/cloud/operators/cloud_memorystore.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/cloud_sql.py b/airflow/providers/google/cloud/operators/cloud_sql.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/cloud_storage_transfer_service.py b/airflow/providers/google/cloud/operators/cloud_storage_transfer_service.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/compute.py b/airflow/providers/google/cloud/operators/compute.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/datacatalog.py b/airflow/providers/google/cloud/operators/datacatalog.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/dataflow.py b/airflow/providers/google/cloud/operators/dataflow.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/datafusion.py b/airflow/providers/google/cloud/operators/datafusion.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/dataprep.py b/airflow/providers/google/cloud/operators/dataprep.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/dataproc.py b/airflow/providers/google/cloud/operators/dataproc.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/datastore.py b/airflow/providers/google/cloud/operators/datastore.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/dlp.py b/airflow/providers/google/cloud/operators/dlp.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/functions.py b/airflow/providers/google/cloud/operators/functions.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/gcs.py b/airflow/providers/google/cloud/operators/gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/kubernetes_engine.py b/airflow/providers/google/cloud/operators/kubernetes_engine.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/life_sciences.py b/airflow/providers/google/cloud/operators/life_sciences.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/mlengine.py b/airflow/providers/google/cloud/operators/mlengine.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/natural_language.py b/airflow/providers/google/cloud/operators/natural_language.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/pubsub.py b/airflow/providers/google/cloud/operators/pubsub.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/spanner.py b/airflow/providers/google/cloud/operators/spanner.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/speech_to_text.py b/airflow/providers/google/cloud/operators/speech_to_text.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/stackdriver.py b/airflow/providers/google/cloud/operators/stackdriver.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/tasks.py b/airflow/providers/google/cloud/operators/tasks.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/text_to_speech.py b/airflow/providers/google/cloud/operators/text_to_speech.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/translate.py b/airflow/providers/google/cloud/operators/translate.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/translate_speech.py b/airflow/providers/google/cloud/operators/translate_speech.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/video_intelligence.py b/airflow/providers/google/cloud/operators/video_intelligence.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/operators/vision.py b/airflow/providers/google/cloud/operators/vision.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/secrets/__init__.py b/airflow/providers/google/cloud/secrets/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/secrets/secret_manager.py b/airflow/providers/google/cloud/secrets/secret_manager.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/sensors/__init__.py b/airflow/providers/google/cloud/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/sensors/bigquery.py b/airflow/providers/google/cloud/sensors/bigquery.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/sensors/bigquery_dts.py b/airflow/providers/google/cloud/sensors/bigquery_dts.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/sensors/bigtable.py b/airflow/providers/google/cloud/sensors/bigtable.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/sensors/cloud_storage_transfer_service.py b/airflow/providers/google/cloud/sensors/cloud_storage_transfer_service.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/sensors/gcs.py b/airflow/providers/google/cloud/sensors/gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/sensors/pubsub.py b/airflow/providers/google/cloud/sensors/pubsub.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/__init__.py b/airflow/providers/google/cloud/transfers/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/adls_to_gcs.py b/airflow/providers/google/cloud/transfers/adls_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/bigquery_to_bigquery.py b/airflow/providers/google/cloud/transfers/bigquery_to_bigquery.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/bigquery_to_gcs.py b/airflow/providers/google/cloud/transfers/bigquery_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/bigquery_to_mysql.py b/airflow/providers/google/cloud/transfers/bigquery_to_mysql.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/cassandra_to_gcs.py b/airflow/providers/google/cloud/transfers/cassandra_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/facebook_ads_to_gcs.py b/airflow/providers/google/cloud/transfers/facebook_ads_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/gcs_to_bigquery.py b/airflow/providers/google/cloud/transfers/gcs_to_bigquery.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/gcs_to_gcs.py b/airflow/providers/google/cloud/transfers/gcs_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/gcs_to_local.py b/airflow/providers/google/cloud/transfers/gcs_to_local.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/gcs_to_sftp.py b/airflow/providers/google/cloud/transfers/gcs_to_sftp.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/local_to_gcs.py b/airflow/providers/google/cloud/transfers/local_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/mssql_to_gcs.py b/airflow/providers/google/cloud/transfers/mssql_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/mysql_to_gcs.py b/airflow/providers/google/cloud/transfers/mysql_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/postgres_to_gcs.py b/airflow/providers/google/cloud/transfers/postgres_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/presto_to_gcs.py b/airflow/providers/google/cloud/transfers/presto_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/s3_to_gcs.py b/airflow/providers/google/cloud/transfers/s3_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/sftp_to_gcs.py b/airflow/providers/google/cloud/transfers/sftp_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/sheets_to_gcs.py b/airflow/providers/google/cloud/transfers/sheets_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/transfers/sql_to_gcs.py b/airflow/providers/google/cloud/transfers/sql_to_gcs.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/utils/__init__.py b/airflow/providers/google/cloud/utils/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/utils/credentials_provider.py b/airflow/providers/google/cloud/utils/credentials_provider.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/utils/field_sanitizer.py b/airflow/providers/google/cloud/utils/field_sanitizer.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/utils/field_validator.py b/airflow/providers/google/cloud/utils/field_validator.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/utils/mlengine_operator_utils.py b/airflow/providers/google/cloud/utils/mlengine_operator_utils.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/cloud/utils/mlengine_prediction_summary.py b/airflow/providers/google/cloud/utils/mlengine_prediction_summary.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/common/__init__.py b/airflow/providers/google/common/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/common/auth_backend/__init__.py b/airflow/providers/google/common/auth_backend/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/common/auth_backend/google_openid.py b/airflow/providers/google/common/auth_backend/google_openid.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/common/hooks/__init__.py b/airflow/providers/google/common/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/common/hooks/base_google.py b/airflow/providers/google/common/hooks/base_google.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/common/hooks/discovery_api.py b/airflow/providers/google/common/hooks/discovery_api.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/common/utils/__init__.py b/airflow/providers/google/common/utils/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/common/utils/id_token_credentials.py b/airflow/providers/google/common/utils/id_token_credentials.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/config_templates/config.yml b/airflow/providers/google/config_templates/config.yml old mode 100644 new mode 100755 diff --git a/airflow/providers/google/config_templates/default_config.cfg b/airflow/providers/google/config_templates/default_config.cfg old mode 100644 new mode 100755 diff --git a/airflow/providers/google/firebase/__init__.py b/airflow/providers/google/firebase/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/firebase/example_dags/__init__.py b/airflow/providers/google/firebase/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/firebase/example_dags/example_firestore.py b/airflow/providers/google/firebase/example_dags/example_firestore.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/firebase/hooks/__init__.py b/airflow/providers/google/firebase/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/firebase/hooks/firestore.py b/airflow/providers/google/firebase/hooks/firestore.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/firebase/operators/__init__.py b/airflow/providers/google/firebase/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/firebase/operators/firestore.py b/airflow/providers/google/firebase/operators/firestore.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/__init__.py b/airflow/providers/google/marketing_platform/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/example_dags/__init__.py b/airflow/providers/google/marketing_platform/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/example_dags/example_analytics.py b/airflow/providers/google/marketing_platform/example_dags/example_analytics.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/example_dags/example_campaign_manager.py b/airflow/providers/google/marketing_platform/example_dags/example_campaign_manager.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/example_dags/example_display_video.py b/airflow/providers/google/marketing_platform/example_dags/example_display_video.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/example_dags/example_search_ads.py b/airflow/providers/google/marketing_platform/example_dags/example_search_ads.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/hooks/__init__.py b/airflow/providers/google/marketing_platform/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/hooks/analytics.py b/airflow/providers/google/marketing_platform/hooks/analytics.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/hooks/campaign_manager.py b/airflow/providers/google/marketing_platform/hooks/campaign_manager.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/hooks/display_video.py b/airflow/providers/google/marketing_platform/hooks/display_video.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/hooks/search_ads.py b/airflow/providers/google/marketing_platform/hooks/search_ads.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/operators/__init__.py b/airflow/providers/google/marketing_platform/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/operators/analytics.py b/airflow/providers/google/marketing_platform/operators/analytics.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/operators/campaign_manager.py b/airflow/providers/google/marketing_platform/operators/campaign_manager.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/operators/display_video.py b/airflow/providers/google/marketing_platform/operators/display_video.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/operators/search_ads.py b/airflow/providers/google/marketing_platform/operators/search_ads.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/sensors/__init__.py b/airflow/providers/google/marketing_platform/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/sensors/campaign_manager.py b/airflow/providers/google/marketing_platform/sensors/campaign_manager.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/sensors/display_video.py b/airflow/providers/google/marketing_platform/sensors/display_video.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/marketing_platform/sensors/search_ads.py b/airflow/providers/google/marketing_platform/sensors/search_ads.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/suite/__init__.py b/airflow/providers/google/suite/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/suite/example_dags/__init__.py b/airflow/providers/google/suite/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/suite/example_dags/example_gcs_to_gdrive.py b/airflow/providers/google/suite/example_dags/example_gcs_to_gdrive.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/suite/example_dags/example_gcs_to_sheets.py b/airflow/providers/google/suite/example_dags/example_gcs_to_sheets.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/suite/example_dags/example_sheets.py b/airflow/providers/google/suite/example_dags/example_sheets.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/suite/hooks/__init__.py b/airflow/providers/google/suite/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/suite/hooks/drive.py b/airflow/providers/google/suite/hooks/drive.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/suite/hooks/sheets.py b/airflow/providers/google/suite/hooks/sheets.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/suite/operators/__init__.py b/airflow/providers/google/suite/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/suite/operators/sheets.py b/airflow/providers/google/suite/operators/sheets.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/suite/transfers/__init__.py b/airflow/providers/google/suite/transfers/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/suite/transfers/gcs_to_gdrive.py b/airflow/providers/google/suite/transfers/gcs_to_gdrive.py old mode 100644 new mode 100755 diff --git a/airflow/providers/google/suite/transfers/gcs_to_sheets.py b/airflow/providers/google/suite/transfers/gcs_to_sheets.py old mode 100644 new mode 100755 diff --git a/airflow/providers/grpc/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/grpc/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/grpc/README.md b/airflow/providers/grpc/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/grpc/__init__.py b/airflow/providers/grpc/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/grpc/hooks/__init__.py b/airflow/providers/grpc/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/grpc/hooks/grpc.py b/airflow/providers/grpc/hooks/grpc.py old mode 100644 new mode 100755 diff --git a/airflow/providers/grpc/operators/__init__.py b/airflow/providers/grpc/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/grpc/operators/grpc.py b/airflow/providers/grpc/operators/grpc.py old mode 100644 new mode 100755 diff --git a/airflow/providers/hashicorp/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/hashicorp/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/hashicorp/README.md b/airflow/providers/hashicorp/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/hashicorp/__init__.py b/airflow/providers/hashicorp/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/hashicorp/_internal_client/__init__.py b/airflow/providers/hashicorp/_internal_client/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/hashicorp/_internal_client/vault_client.py b/airflow/providers/hashicorp/_internal_client/vault_client.py old mode 100644 new mode 100755 diff --git a/airflow/providers/hashicorp/hooks/__init__.py b/airflow/providers/hashicorp/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/hashicorp/hooks/vault.py b/airflow/providers/hashicorp/hooks/vault.py old mode 100644 new mode 100755 diff --git a/airflow/providers/hashicorp/secrets/__init__.py b/airflow/providers/hashicorp/secrets/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/hashicorp/secrets/vault.py b/airflow/providers/hashicorp/secrets/vault.py old mode 100644 new mode 100755 diff --git a/airflow/providers/http/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/http/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/http/README.md b/airflow/providers/http/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/http/__init__.py b/airflow/providers/http/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/http/example_dags/__init__.py b/airflow/providers/http/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/http/example_dags/example_http.py b/airflow/providers/http/example_dags/example_http.py old mode 100644 new mode 100755 diff --git a/airflow/providers/http/hooks/__init__.py b/airflow/providers/http/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/http/hooks/http.py b/airflow/providers/http/hooks/http.py old mode 100644 new mode 100755 diff --git a/airflow/providers/http/operators/__init__.py b/airflow/providers/http/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/http/operators/http.py b/airflow/providers/http/operators/http.py old mode 100644 new mode 100755 diff --git a/airflow/providers/http/sensors/__init__.py b/airflow/providers/http/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/http/sensors/http.py b/airflow/providers/http/sensors/http.py old mode 100644 new mode 100755 diff --git a/airflow/providers/imap/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/imap/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/imap/README.md b/airflow/providers/imap/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/imap/__init__.py b/airflow/providers/imap/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/imap/hooks/__init__.py b/airflow/providers/imap/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/imap/hooks/imap.py b/airflow/providers/imap/hooks/imap.py old mode 100644 new mode 100755 diff --git a/airflow/providers/imap/sensors/__init__.py b/airflow/providers/imap/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/imap/sensors/imap_attachment.py b/airflow/providers/imap/sensors/imap_attachment.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jdbc/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/jdbc/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/jdbc/README.md b/airflow/providers/jdbc/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/jdbc/__init__.py b/airflow/providers/jdbc/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jdbc/hooks/__init__.py b/airflow/providers/jdbc/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jdbc/hooks/jdbc.py b/airflow/providers/jdbc/hooks/jdbc.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jdbc/operators/__init__.py b/airflow/providers/jdbc/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jdbc/operators/jdbc.py b/airflow/providers/jdbc/operators/jdbc.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jenkins/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/jenkins/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/jenkins/README.md b/airflow/providers/jenkins/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/jenkins/__init__.py b/airflow/providers/jenkins/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jenkins/example_dags/__init__.py b/airflow/providers/jenkins/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jenkins/example_dags/example_jenkins_job_trigger.py b/airflow/providers/jenkins/example_dags/example_jenkins_job_trigger.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jenkins/hooks/__init__.py b/airflow/providers/jenkins/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jenkins/hooks/jenkins.py b/airflow/providers/jenkins/hooks/jenkins.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jenkins/operators/__init__.py b/airflow/providers/jenkins/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jenkins/operators/jenkins_job_trigger.py b/airflow/providers/jenkins/operators/jenkins_job_trigger.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jira/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/jira/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/jira/README.md b/airflow/providers/jira/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/jira/__init__.py b/airflow/providers/jira/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jira/hooks/__init__.py b/airflow/providers/jira/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jira/hooks/jira.py b/airflow/providers/jira/hooks/jira.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jira/operators/__init__.py b/airflow/providers/jira/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jira/operators/jira.py b/airflow/providers/jira/operators/jira.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jira/sensors/__init__.py b/airflow/providers/jira/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/jira/sensors/jira.py b/airflow/providers/jira/sensors/jira.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/__init__.py b/airflow/providers/microsoft/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/microsoft/azure/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/README.md b/airflow/providers/microsoft/azure/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/__init__.py b/airflow/providers/microsoft/azure/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/example_dags/__init__.py b/airflow/providers/microsoft/azure/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/example_dags/example_azure_container_instances.py b/airflow/providers/microsoft/azure/example_dags/example_azure_container_instances.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/example_dags/example_azure_cosmosdb.py b/airflow/providers/microsoft/azure/example_dags/example_azure_cosmosdb.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/hooks/__init__.py b/airflow/providers/microsoft/azure/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/hooks/adx.py b/airflow/providers/microsoft/azure/hooks/adx.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/hooks/azure_batch.py b/airflow/providers/microsoft/azure/hooks/azure_batch.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/hooks/azure_container_instance.py b/airflow/providers/microsoft/azure/hooks/azure_container_instance.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/hooks/azure_container_registry.py b/airflow/providers/microsoft/azure/hooks/azure_container_registry.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/hooks/azure_container_volume.py b/airflow/providers/microsoft/azure/hooks/azure_container_volume.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/hooks/azure_cosmos.py b/airflow/providers/microsoft/azure/hooks/azure_cosmos.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/hooks/azure_data_lake.py b/airflow/providers/microsoft/azure/hooks/azure_data_lake.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/hooks/azure_fileshare.py b/airflow/providers/microsoft/azure/hooks/azure_fileshare.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/hooks/base_azure.py b/airflow/providers/microsoft/azure/hooks/base_azure.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/hooks/wasb.py b/airflow/providers/microsoft/azure/hooks/wasb.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/log/__init__.py b/airflow/providers/microsoft/azure/log/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/log/wasb_task_handler.py b/airflow/providers/microsoft/azure/log/wasb_task_handler.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/operators/__init__.py b/airflow/providers/microsoft/azure/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/operators/adls_list.py b/airflow/providers/microsoft/azure/operators/adls_list.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/operators/adx.py b/airflow/providers/microsoft/azure/operators/adx.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/operators/azure_batch.py b/airflow/providers/microsoft/azure/operators/azure_batch.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/operators/azure_container_instances.py b/airflow/providers/microsoft/azure/operators/azure_container_instances.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/operators/azure_cosmos.py b/airflow/providers/microsoft/azure/operators/azure_cosmos.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/operators/wasb_delete_blob.py b/airflow/providers/microsoft/azure/operators/wasb_delete_blob.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/sensors/__init__.py b/airflow/providers/microsoft/azure/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/sensors/azure_cosmos.py b/airflow/providers/microsoft/azure/sensors/azure_cosmos.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/sensors/wasb.py b/airflow/providers/microsoft/azure/sensors/wasb.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/transfers/__init__.py b/airflow/providers/microsoft/azure/transfers/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/transfers/file_to_wasb.py b/airflow/providers/microsoft/azure/transfers/file_to_wasb.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/azure/transfers/oracle_to_azure_data_lake.py b/airflow/providers/microsoft/azure/transfers/oracle_to_azure_data_lake.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/mssql/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/microsoft/mssql/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/mssql/README.md b/airflow/providers/microsoft/mssql/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/mssql/__init__.py b/airflow/providers/microsoft/mssql/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/mssql/hooks/__init__.py b/airflow/providers/microsoft/mssql/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/mssql/hooks/mssql.py b/airflow/providers/microsoft/mssql/hooks/mssql.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/mssql/operators/__init__.py b/airflow/providers/microsoft/mssql/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/mssql/operators/mssql.py b/airflow/providers/microsoft/mssql/operators/mssql.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/winrm/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/microsoft/winrm/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/winrm/README.md b/airflow/providers/microsoft/winrm/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/winrm/__init__.py b/airflow/providers/microsoft/winrm/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/winrm/example_dags/__init__.py b/airflow/providers/microsoft/winrm/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/winrm/example_dags/example_winrm.py b/airflow/providers/microsoft/winrm/example_dags/example_winrm.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/winrm/hooks/__init__.py b/airflow/providers/microsoft/winrm/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/winrm/hooks/winrm.py b/airflow/providers/microsoft/winrm/hooks/winrm.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/winrm/operators/__init__.py b/airflow/providers/microsoft/winrm/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/microsoft/winrm/operators/winrm.py b/airflow/providers/microsoft/winrm/operators/winrm.py old mode 100644 new mode 100755 diff --git a/airflow/providers/mongo/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/mongo/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/mongo/README.md b/airflow/providers/mongo/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/mongo/__init__.py b/airflow/providers/mongo/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/mongo/hooks/__init__.py b/airflow/providers/mongo/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/mongo/hooks/mongo.py b/airflow/providers/mongo/hooks/mongo.py old mode 100644 new mode 100755 diff --git a/airflow/providers/mongo/sensors/__init__.py b/airflow/providers/mongo/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/mongo/sensors/mongo.py b/airflow/providers/mongo/sensors/mongo.py old mode 100644 new mode 100755 diff --git a/airflow/providers/mysql/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/mysql/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/mysql/README.md b/airflow/providers/mysql/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/mysql/__init__.py b/airflow/providers/mysql/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/mysql/hooks/__init__.py b/airflow/providers/mysql/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/mysql/hooks/mysql.py b/airflow/providers/mysql/hooks/mysql.py old mode 100644 new mode 100755 diff --git a/airflow/providers/mysql/operators/__init__.py b/airflow/providers/mysql/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/mysql/operators/mysql.py b/airflow/providers/mysql/operators/mysql.py old mode 100644 new mode 100755 diff --git a/airflow/providers/mysql/transfers/__init__.py b/airflow/providers/mysql/transfers/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/mysql/transfers/presto_to_mysql.py b/airflow/providers/mysql/transfers/presto_to_mysql.py old mode 100644 new mode 100755 diff --git a/airflow/providers/mysql/transfers/s3_to_mysql.py b/airflow/providers/mysql/transfers/s3_to_mysql.py old mode 100644 new mode 100755 diff --git a/airflow/providers/mysql/transfers/vertica_to_mysql.py b/airflow/providers/mysql/transfers/vertica_to_mysql.py old mode 100644 new mode 100755 diff --git a/airflow/providers/odbc/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/odbc/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/odbc/README.md b/airflow/providers/odbc/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/odbc/__init__.py b/airflow/providers/odbc/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/odbc/hooks/__init__.py b/airflow/providers/odbc/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/odbc/hooks/odbc.py b/airflow/providers/odbc/hooks/odbc.py old mode 100644 new mode 100755 diff --git a/airflow/providers/openfaas/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/openfaas/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/openfaas/README.md b/airflow/providers/openfaas/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/openfaas/__init__.py b/airflow/providers/openfaas/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/openfaas/hooks/__init__.py b/airflow/providers/openfaas/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/openfaas/hooks/openfaas.py b/airflow/providers/openfaas/hooks/openfaas.py old mode 100644 new mode 100755 diff --git a/airflow/providers/opsgenie/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/opsgenie/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/opsgenie/README.md b/airflow/providers/opsgenie/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/opsgenie/__init__.py b/airflow/providers/opsgenie/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/opsgenie/hooks/__init__.py b/airflow/providers/opsgenie/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/opsgenie/hooks/opsgenie_alert.py b/airflow/providers/opsgenie/hooks/opsgenie_alert.py old mode 100644 new mode 100755 diff --git a/airflow/providers/opsgenie/operators/__init__.py b/airflow/providers/opsgenie/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/opsgenie/operators/opsgenie_alert.py b/airflow/providers/opsgenie/operators/opsgenie_alert.py old mode 100644 new mode 100755 diff --git a/airflow/providers/oracle/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/oracle/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/oracle/README.md b/airflow/providers/oracle/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/oracle/__init__.py b/airflow/providers/oracle/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/oracle/hooks/__init__.py b/airflow/providers/oracle/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/oracle/hooks/oracle.py b/airflow/providers/oracle/hooks/oracle.py old mode 100644 new mode 100755 diff --git a/airflow/providers/oracle/operators/__init__.py b/airflow/providers/oracle/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/oracle/operators/oracle.py b/airflow/providers/oracle/operators/oracle.py old mode 100644 new mode 100755 diff --git a/airflow/providers/oracle/transfers/__init__.py b/airflow/providers/oracle/transfers/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/oracle/transfers/oracle_to_oracle.py b/airflow/providers/oracle/transfers/oracle_to_oracle.py old mode 100644 new mode 100755 diff --git a/airflow/providers/pagerduty/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/pagerduty/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/pagerduty/README.md b/airflow/providers/pagerduty/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/pagerduty/__init__.py b/airflow/providers/pagerduty/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/pagerduty/hooks/__init__.py b/airflow/providers/pagerduty/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/pagerduty/hooks/pagerduty.py b/airflow/providers/pagerduty/hooks/pagerduty.py old mode 100644 new mode 100755 diff --git a/airflow/providers/papermill/ADDITIONAL_INFO.md b/airflow/providers/papermill/ADDITIONAL_INFO.md old mode 100644 new mode 100755 diff --git a/airflow/providers/papermill/__init__.py b/airflow/providers/papermill/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/papermill/example_dags/__init__.py b/airflow/providers/papermill/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/papermill/example_dags/example_papermill.py b/airflow/providers/papermill/example_dags/example_papermill.py old mode 100644 new mode 100755 diff --git a/airflow/providers/papermill/operators/__init__.py b/airflow/providers/papermill/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/papermill/operators/papermill.py b/airflow/providers/papermill/operators/papermill.py old mode 100644 new mode 100755 diff --git a/airflow/providers/postgres/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/postgres/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/postgres/README.md b/airflow/providers/postgres/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/postgres/__init__.py b/airflow/providers/postgres/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/postgres/hooks/__init__.py b/airflow/providers/postgres/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/postgres/hooks/postgres.py b/airflow/providers/postgres/hooks/postgres.py old mode 100644 new mode 100755 diff --git a/airflow/providers/postgres/operators/__init__.py b/airflow/providers/postgres/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/postgres/operators/postgres.py b/airflow/providers/postgres/operators/postgres.py old mode 100644 new mode 100755 diff --git a/airflow/providers/presto/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/presto/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/presto/README.md b/airflow/providers/presto/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/presto/__init__.py b/airflow/providers/presto/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/presto/hooks/__init__.py b/airflow/providers/presto/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/presto/hooks/presto.py b/airflow/providers/presto/hooks/presto.py old mode 100644 new mode 100755 diff --git a/airflow/providers/qubole/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/qubole/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/qubole/README.md b/airflow/providers/qubole/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/qubole/__init__.py b/airflow/providers/qubole/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/qubole/example_dags/__init__.py b/airflow/providers/qubole/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/qubole/example_dags/example_qubole.py b/airflow/providers/qubole/example_dags/example_qubole.py old mode 100644 new mode 100755 diff --git a/airflow/providers/qubole/hooks/__init__.py b/airflow/providers/qubole/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/qubole/hooks/qubole.py b/airflow/providers/qubole/hooks/qubole.py old mode 100644 new mode 100755 diff --git a/airflow/providers/qubole/hooks/qubole_check.py b/airflow/providers/qubole/hooks/qubole_check.py old mode 100644 new mode 100755 diff --git a/airflow/providers/qubole/operators/__init__.py b/airflow/providers/qubole/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/qubole/operators/qubole.py b/airflow/providers/qubole/operators/qubole.py old mode 100644 new mode 100755 diff --git a/airflow/providers/qubole/operators/qubole_check.py b/airflow/providers/qubole/operators/qubole_check.py old mode 100644 new mode 100755 diff --git a/airflow/providers/qubole/sensors/__init__.py b/airflow/providers/qubole/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/qubole/sensors/qubole.py b/airflow/providers/qubole/sensors/qubole.py old mode 100644 new mode 100755 diff --git a/airflow/providers/redis/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/redis/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/redis/README.md b/airflow/providers/redis/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/redis/__init__.py b/airflow/providers/redis/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/redis/hooks/__init__.py b/airflow/providers/redis/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/redis/hooks/redis.py b/airflow/providers/redis/hooks/redis.py old mode 100644 new mode 100755 diff --git a/airflow/providers/redis/operators/__init__.py b/airflow/providers/redis/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/redis/operators/redis_publish.py b/airflow/providers/redis/operators/redis_publish.py old mode 100644 new mode 100755 diff --git a/airflow/providers/redis/sensors/__init__.py b/airflow/providers/redis/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/redis/sensors/redis_key.py b/airflow/providers/redis/sensors/redis_key.py old mode 100644 new mode 100755 diff --git a/airflow/providers/redis/sensors/redis_pub_sub.py b/airflow/providers/redis/sensors/redis_pub_sub.py old mode 100644 new mode 100755 diff --git a/airflow/providers/salesforce/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/salesforce/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/salesforce/README.md b/airflow/providers/salesforce/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/salesforce/__init__.py b/airflow/providers/salesforce/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/salesforce/example_dags/__init__.py b/airflow/providers/salesforce/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/salesforce/example_dags/example_tableau_refresh_workbook.py b/airflow/providers/salesforce/example_dags/example_tableau_refresh_workbook.py old mode 100644 new mode 100755 diff --git a/airflow/providers/salesforce/hooks/__init__.py b/airflow/providers/salesforce/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/salesforce/hooks/salesforce.py b/airflow/providers/salesforce/hooks/salesforce.py old mode 100644 new mode 100755 diff --git a/airflow/providers/salesforce/hooks/tableau.py b/airflow/providers/salesforce/hooks/tableau.py old mode 100644 new mode 100755 diff --git a/airflow/providers/salesforce/operators/__init__.py b/airflow/providers/salesforce/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/salesforce/operators/tableau_refresh_workbook.py b/airflow/providers/salesforce/operators/tableau_refresh_workbook.py old mode 100644 new mode 100755 diff --git a/airflow/providers/salesforce/sensors/__init__.py b/airflow/providers/salesforce/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/salesforce/sensors/tableau_job_status.py b/airflow/providers/salesforce/sensors/tableau_job_status.py old mode 100644 new mode 100755 diff --git a/airflow/providers/samba/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/samba/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/samba/README.md b/airflow/providers/samba/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/samba/__init__.py b/airflow/providers/samba/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/samba/hooks/__init__.py b/airflow/providers/samba/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/samba/hooks/samba.py b/airflow/providers/samba/hooks/samba.py old mode 100644 new mode 100755 diff --git a/airflow/providers/segment/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/segment/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/segment/README.md b/airflow/providers/segment/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/segment/__init__.py b/airflow/providers/segment/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/segment/hooks/__init__.py b/airflow/providers/segment/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/segment/hooks/segment.py b/airflow/providers/segment/hooks/segment.py old mode 100644 new mode 100755 diff --git a/airflow/providers/segment/operators/__init__.py b/airflow/providers/segment/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/segment/operators/segment_track_event.py b/airflow/providers/segment/operators/segment_track_event.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sendgrid/__init__.py b/airflow/providers/sendgrid/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sendgrid/utils/__init__.py b/airflow/providers/sendgrid/utils/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sendgrid/utils/emailer.py b/airflow/providers/sendgrid/utils/emailer.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sftp/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/sftp/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/sftp/README.md b/airflow/providers/sftp/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/sftp/__init__.py b/airflow/providers/sftp/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sftp/hooks/__init__.py b/airflow/providers/sftp/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sftp/hooks/sftp.py b/airflow/providers/sftp/hooks/sftp.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sftp/operators/__init__.py b/airflow/providers/sftp/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sftp/operators/sftp.py b/airflow/providers/sftp/operators/sftp.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sftp/sensors/__init__.py b/airflow/providers/sftp/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sftp/sensors/sftp.py b/airflow/providers/sftp/sensors/sftp.py old mode 100644 new mode 100755 diff --git a/airflow/providers/singularity/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/singularity/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/singularity/README.md b/airflow/providers/singularity/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/singularity/__init__.py b/airflow/providers/singularity/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/singularity/example_dags/__init__.py b/airflow/providers/singularity/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/singularity/example_dags/example_singularity.py b/airflow/providers/singularity/example_dags/example_singularity.py old mode 100644 new mode 100755 diff --git a/airflow/providers/singularity/operators/__init__.py b/airflow/providers/singularity/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/singularity/operators/singularity.py b/airflow/providers/singularity/operators/singularity.py old mode 100644 new mode 100755 diff --git a/airflow/providers/slack/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/slack/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/slack/README.md b/airflow/providers/slack/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/slack/__init__.py b/airflow/providers/slack/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/slack/hooks/__init__.py b/airflow/providers/slack/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/slack/hooks/slack.py b/airflow/providers/slack/hooks/slack.py old mode 100644 new mode 100755 diff --git a/airflow/providers/slack/hooks/slack_webhook.py b/airflow/providers/slack/hooks/slack_webhook.py old mode 100644 new mode 100755 diff --git a/airflow/providers/slack/operators/__init__.py b/airflow/providers/slack/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/slack/operators/slack.py b/airflow/providers/slack/operators/slack.py old mode 100644 new mode 100755 diff --git a/airflow/providers/slack/operators/slack_webhook.py b/airflow/providers/slack/operators/slack_webhook.py old mode 100644 new mode 100755 diff --git a/airflow/providers/snowflake/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/snowflake/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/snowflake/README.md b/airflow/providers/snowflake/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/snowflake/__init__.py b/airflow/providers/snowflake/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/snowflake/example_dags/__init__.py b/airflow/providers/snowflake/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/snowflake/example_dags/example_snowflake.py b/airflow/providers/snowflake/example_dags/example_snowflake.py old mode 100644 new mode 100755 diff --git a/airflow/providers/snowflake/hooks/__init__.py b/airflow/providers/snowflake/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/snowflake/hooks/snowflake.py b/airflow/providers/snowflake/hooks/snowflake.py old mode 100644 new mode 100755 diff --git a/airflow/providers/snowflake/operators/__init__.py b/airflow/providers/snowflake/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/snowflake/operators/snowflake.py b/airflow/providers/snowflake/operators/snowflake.py old mode 100644 new mode 100755 diff --git a/airflow/providers/snowflake/transfers/__init__.py b/airflow/providers/snowflake/transfers/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/snowflake/transfers/s3_to_snowflake.py b/airflow/providers/snowflake/transfers/s3_to_snowflake.py old mode 100644 new mode 100755 diff --git a/airflow/providers/snowflake/transfers/snowflake_to_slack.py b/airflow/providers/snowflake/transfers/snowflake_to_slack.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sqlite/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/sqlite/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/sqlite/README.md b/airflow/providers/sqlite/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/sqlite/__init__.py b/airflow/providers/sqlite/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sqlite/hooks/__init__.py b/airflow/providers/sqlite/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sqlite/hooks/sqlite.py b/airflow/providers/sqlite/hooks/sqlite.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sqlite/operators/__init__.py b/airflow/providers/sqlite/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/sqlite/operators/sqlite.py b/airflow/providers/sqlite/operators/sqlite.py old mode 100644 new mode 100755 diff --git a/airflow/providers/ssh/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/ssh/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/ssh/README.md b/airflow/providers/ssh/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/ssh/__init__.py b/airflow/providers/ssh/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/ssh/hooks/__init__.py b/airflow/providers/ssh/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/ssh/hooks/ssh.py b/airflow/providers/ssh/hooks/ssh.py old mode 100644 new mode 100755 diff --git a/airflow/providers/ssh/operators/__init__.py b/airflow/providers/ssh/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/ssh/operators/ssh.py b/airflow/providers/ssh/operators/ssh.py old mode 100644 new mode 100755 diff --git a/airflow/providers/vertica/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/vertica/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/vertica/README.md b/airflow/providers/vertica/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/vertica/__init__.py b/airflow/providers/vertica/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/vertica/hooks/__init__.py b/airflow/providers/vertica/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/vertica/hooks/vertica.py b/airflow/providers/vertica/hooks/vertica.py old mode 100644 new mode 100755 diff --git a/airflow/providers/vertica/operators/__init__.py b/airflow/providers/vertica/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/vertica/operators/vertica.py b/airflow/providers/vertica/operators/vertica.py old mode 100644 new mode 100755 diff --git a/airflow/providers/yandex/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/yandex/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/yandex/README.md b/airflow/providers/yandex/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/yandex/__init__.py b/airflow/providers/yandex/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/yandex/example_dags/__init__.py b/airflow/providers/yandex/example_dags/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/yandex/example_dags/example_yandexcloud_dataproc.py b/airflow/providers/yandex/example_dags/example_yandexcloud_dataproc.py old mode 100644 new mode 100755 diff --git a/airflow/providers/yandex/hooks/__init__.py b/airflow/providers/yandex/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/yandex/hooks/yandex.py b/airflow/providers/yandex/hooks/yandex.py old mode 100644 new mode 100755 diff --git a/airflow/providers/yandex/hooks/yandexcloud_dataproc.py b/airflow/providers/yandex/hooks/yandexcloud_dataproc.py old mode 100644 new mode 100755 diff --git a/airflow/providers/yandex/operators/__init__.py b/airflow/providers/yandex/operators/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/yandex/operators/yandexcloud_dataproc.py b/airflow/providers/yandex/operators/yandexcloud_dataproc.py old mode 100644 new mode 100755 diff --git a/airflow/providers/zendesk/PROVIDERS_CHANGES_2020.06.24.md b/airflow/providers/zendesk/PROVIDERS_CHANGES_2020.06.24.md old mode 100644 new mode 100755 diff --git a/airflow/providers/zendesk/README.md b/airflow/providers/zendesk/README.md old mode 100644 new mode 100755 diff --git a/airflow/providers/zendesk/__init__.py b/airflow/providers/zendesk/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/zendesk/hooks/__init__.py b/airflow/providers/zendesk/hooks/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/providers/zendesk/hooks/zendesk.py b/airflow/providers/zendesk/hooks/zendesk.py old mode 100644 new mode 100755 diff --git a/airflow/py.typed b/airflow/py.typed old mode 100644 new mode 100755 diff --git a/airflow/secrets/__init__.py b/airflow/secrets/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/secrets/base_secrets.py b/airflow/secrets/base_secrets.py old mode 100644 new mode 100755 diff --git a/airflow/secrets/environment_variables.py b/airflow/secrets/environment_variables.py old mode 100644 new mode 100755 diff --git a/airflow/secrets/local_filesystem.py b/airflow/secrets/local_filesystem.py old mode 100644 new mode 100755 diff --git a/airflow/secrets/metastore.py b/airflow/secrets/metastore.py old mode 100644 new mode 100755 diff --git a/airflow/security/__init__.py b/airflow/security/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/security/kerberos.py b/airflow/security/kerberos.py old mode 100644 new mode 100755 diff --git a/airflow/security/utils.py b/airflow/security/utils.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/__init__.py b/airflow/sensors/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/base_sensor_operator.py b/airflow/sensors/base_sensor_operator.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/bash.py b/airflow/sensors/bash.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/date_time_sensor.py b/airflow/sensors/date_time_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/external_task_sensor.py b/airflow/sensors/external_task_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/filesystem.py b/airflow/sensors/filesystem.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/hdfs_sensor.py b/airflow/sensors/hdfs_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/hive_partition_sensor.py b/airflow/sensors/hive_partition_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/http_sensor.py b/airflow/sensors/http_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/metastore_partition_sensor.py b/airflow/sensors/metastore_partition_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/named_hive_partition_sensor.py b/airflow/sensors/named_hive_partition_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/python.py b/airflow/sensors/python.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/s3_key_sensor.py b/airflow/sensors/s3_key_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/s3_prefix_sensor.py b/airflow/sensors/s3_prefix_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/sql_sensor.py b/airflow/sensors/sql_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/time_delta_sensor.py b/airflow/sensors/time_delta_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/time_sensor.py b/airflow/sensors/time_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/web_hdfs_sensor.py b/airflow/sensors/web_hdfs_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/sensors/weekday_sensor.py b/airflow/sensors/weekday_sensor.py old mode 100644 new mode 100755 diff --git a/airflow/sentry.py b/airflow/sentry.py old mode 100644 new mode 100755 diff --git a/airflow/serialization/__init__.py b/airflow/serialization/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/serialization/enums.py b/airflow/serialization/enums.py old mode 100644 new mode 100755 diff --git a/airflow/serialization/helpers.py b/airflow/serialization/helpers.py old mode 100644 new mode 100755 diff --git a/airflow/serialization/json_schema.py b/airflow/serialization/json_schema.py old mode 100644 new mode 100755 diff --git a/airflow/serialization/schema.json b/airflow/serialization/schema.json old mode 100644 new mode 100755 diff --git a/airflow/serialization/serialized_objects.py b/airflow/serialization/serialized_objects.py old mode 100644 new mode 100755 diff --git a/airflow/settings.py b/airflow/settings.py old mode 100644 new mode 100755 diff --git a/airflow/stats.py b/airflow/stats.py old mode 100644 new mode 100755 diff --git a/airflow/task/__init__.py b/airflow/task/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/task/task_runner/__init__.py b/airflow/task/task_runner/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/task/task_runner/base_task_runner.py b/airflow/task/task_runner/base_task_runner.py old mode 100644 new mode 100755 diff --git a/airflow/task/task_runner/cgroup_task_runner.py b/airflow/task/task_runner/cgroup_task_runner.py old mode 100644 new mode 100755 diff --git a/airflow/task/task_runner/standard_task_runner.py b/airflow/task/task_runner/standard_task_runner.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/__init__.py b/airflow/ti_deps/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/dep_context.py b/airflow/ti_deps/dep_context.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/dependencies_deps.py b/airflow/ti_deps/dependencies_deps.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/dependencies_states.py b/airflow/ti_deps/dependencies_states.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/__init__.py b/airflow/ti_deps/deps/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/base_ti_dep.py b/airflow/ti_deps/deps/base_ti_dep.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/dag_ti_slots_available_dep.py b/airflow/ti_deps/deps/dag_ti_slots_available_dep.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/dag_unpaused_dep.py b/airflow/ti_deps/deps/dag_unpaused_dep.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/dagrun_exists_dep.py b/airflow/ti_deps/deps/dagrun_exists_dep.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/dagrun_id_dep.py b/airflow/ti_deps/deps/dagrun_id_dep.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/exec_date_after_start_date_dep.py b/airflow/ti_deps/deps/exec_date_after_start_date_dep.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/not_in_retry_period_dep.py b/airflow/ti_deps/deps/not_in_retry_period_dep.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/not_previously_skipped_dep.py b/airflow/ti_deps/deps/not_previously_skipped_dep.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/pool_slots_available_dep.py b/airflow/ti_deps/deps/pool_slots_available_dep.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/prev_dagrun_dep.py b/airflow/ti_deps/deps/prev_dagrun_dep.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/ready_to_reschedule.py b/airflow/ti_deps/deps/ready_to_reschedule.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/runnable_exec_date_dep.py b/airflow/ti_deps/deps/runnable_exec_date_dep.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/task_concurrency_dep.py b/airflow/ti_deps/deps/task_concurrency_dep.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/task_not_running_dep.py b/airflow/ti_deps/deps/task_not_running_dep.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/trigger_rule_dep.py b/airflow/ti_deps/deps/trigger_rule_dep.py old mode 100644 new mode 100755 diff --git a/airflow/ti_deps/deps/valid_state_dep.py b/airflow/ti_deps/deps/valid_state_dep.py old mode 100644 new mode 100755 diff --git a/airflow/typing_compat.py b/airflow/typing_compat.py old mode 100644 new mode 100755 diff --git a/airflow/utils/__init__.py b/airflow/utils/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/utils/asciiart.py b/airflow/utils/asciiart.py old mode 100644 new mode 100755 diff --git a/airflow/utils/cli.py b/airflow/utils/cli.py old mode 100644 new mode 100755 diff --git a/airflow/utils/cli_action_loggers.py b/airflow/utils/cli_action_loggers.py old mode 100644 new mode 100755 diff --git a/airflow/utils/code_utils.py b/airflow/utils/code_utils.py old mode 100644 new mode 100755 diff --git a/airflow/utils/compression.py b/airflow/utils/compression.py old mode 100644 new mode 100755 diff --git a/airflow/utils/configuration.py b/airflow/utils/configuration.py old mode 100644 new mode 100755 diff --git a/airflow/utils/dag_cycle_tester.py b/airflow/utils/dag_cycle_tester.py old mode 100644 new mode 100755 diff --git a/airflow/utils/dag_processing.py b/airflow/utils/dag_processing.py old mode 100644 new mode 100755 diff --git a/airflow/utils/dates.py b/airflow/utils/dates.py old mode 100644 new mode 100755 diff --git a/airflow/utils/db.py b/airflow/utils/db.py old mode 100644 new mode 100755 diff --git a/airflow/utils/decorators.py b/airflow/utils/decorators.py old mode 100644 new mode 100755 diff --git a/airflow/utils/docs.py b/airflow/utils/docs.py old mode 100644 new mode 100755 diff --git a/airflow/utils/dot_renderer.py b/airflow/utils/dot_renderer.py old mode 100644 new mode 100755 diff --git a/airflow/utils/email.py b/airflow/utils/email.py old mode 100644 new mode 100755 diff --git a/airflow/utils/file.py b/airflow/utils/file.py old mode 100644 new mode 100755 diff --git a/airflow/utils/helpers.py b/airflow/utils/helpers.py old mode 100644 new mode 100755 diff --git a/airflow/utils/json.py b/airflow/utils/json.py old mode 100644 new mode 100755 diff --git a/airflow/utils/log/__init__.py b/airflow/utils/log/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/utils/log/cloudwatch_task_handler.py b/airflow/utils/log/cloudwatch_task_handler.py old mode 100644 new mode 100755 diff --git a/airflow/utils/log/colored_log.py b/airflow/utils/log/colored_log.py old mode 100644 new mode 100755 diff --git a/airflow/utils/log/es_task_handler.py b/airflow/utils/log/es_task_handler.py old mode 100644 new mode 100755 diff --git a/airflow/utils/log/file_processor_handler.py b/airflow/utils/log/file_processor_handler.py old mode 100644 new mode 100755 diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py old mode 100644 new mode 100755 diff --git a/airflow/utils/log/gcs_task_handler.py b/airflow/utils/log/gcs_task_handler.py old mode 100644 new mode 100755 diff --git a/airflow/utils/log/json_formatter.py b/airflow/utils/log/json_formatter.py old mode 100644 new mode 100755 diff --git a/airflow/utils/log/log_reader.py b/airflow/utils/log/log_reader.py old mode 100644 new mode 100755 diff --git a/airflow/utils/log/logging_mixin.py b/airflow/utils/log/logging_mixin.py old mode 100644 new mode 100755 diff --git a/airflow/utils/log/s3_task_handler.py b/airflow/utils/log/s3_task_handler.py old mode 100644 new mode 100755 diff --git a/airflow/utils/log/stackdriver_task_handler.py b/airflow/utils/log/stackdriver_task_handler.py old mode 100644 new mode 100755 diff --git a/airflow/utils/log/task_handler_with_custom_formatter.py b/airflow/utils/log/task_handler_with_custom_formatter.py old mode 100644 new mode 100755 diff --git a/airflow/utils/log/wasb_task_handler.py b/airflow/utils/log/wasb_task_handler.py old mode 100644 new mode 100755 diff --git a/airflow/utils/mixins.py b/airflow/utils/mixins.py old mode 100644 new mode 100755 diff --git a/airflow/utils/module_loading.py b/airflow/utils/module_loading.py old mode 100644 new mode 100755 diff --git a/airflow/utils/net.py b/airflow/utils/net.py old mode 100644 new mode 100755 diff --git a/airflow/utils/operator_helpers.py b/airflow/utils/operator_helpers.py old mode 100644 new mode 100755 diff --git a/airflow/utils/operator_resources.py b/airflow/utils/operator_resources.py old mode 100644 new mode 100755 diff --git a/airflow/utils/orm_event_handlers.py b/airflow/utils/orm_event_handlers.py old mode 100644 new mode 100755 diff --git a/airflow/utils/platform.py b/airflow/utils/platform.py old mode 100644 new mode 100755 diff --git a/airflow/utils/process_utils.py b/airflow/utils/process_utils.py old mode 100644 new mode 100755 diff --git a/airflow/utils/python_virtualenv.py b/airflow/utils/python_virtualenv.py old mode 100644 new mode 100755 diff --git a/airflow/utils/python_virtualenv_script.jinja2 b/airflow/utils/python_virtualenv_script.jinja2 old mode 100644 new mode 100755 diff --git a/airflow/utils/serve_logs.py b/airflow/utils/serve_logs.py old mode 100644 new mode 100755 diff --git a/airflow/utils/session.py b/airflow/utils/session.py old mode 100644 new mode 100755 diff --git a/airflow/utils/sqlalchemy.py b/airflow/utils/sqlalchemy.py old mode 100644 new mode 100755 diff --git a/airflow/utils/state.py b/airflow/utils/state.py old mode 100644 new mode 100755 diff --git a/airflow/utils/strings.py b/airflow/utils/strings.py old mode 100644 new mode 100755 diff --git a/airflow/utils/timeout.py b/airflow/utils/timeout.py old mode 100644 new mode 100755 diff --git a/airflow/utils/timezone.py b/airflow/utils/timezone.py old mode 100644 new mode 100755 diff --git a/airflow/utils/trigger_rule.py b/airflow/utils/trigger_rule.py old mode 100644 new mode 100755 diff --git a/airflow/utils/types.py b/airflow/utils/types.py old mode 100644 new mode 100755 diff --git a/airflow/utils/weekday.py b/airflow/utils/weekday.py old mode 100644 new mode 100755 diff --git a/airflow/utils/weight_rule.py b/airflow/utils/weight_rule.py old mode 100644 new mode 100755 diff --git a/airflow/version.py b/airflow/version.py old mode 100644 new mode 100755 diff --git a/airflow/www/.eslintignore b/airflow/www/.eslintignore old mode 100644 new mode 100755 diff --git a/airflow/www/.eslintrc b/airflow/www/.eslintrc old mode 100644 new mode 100755 diff --git a/airflow/www/.stylelintignore b/airflow/www/.stylelintignore old mode 100644 new mode 100755 diff --git a/airflow/www/.stylelintrc b/airflow/www/.stylelintrc old mode 100644 new mode 100755 diff --git a/airflow/www/__init__.py b/airflow/www/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/www/api/__init__.py b/airflow/www/api/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/www/api/experimental/__init__.py b/airflow/www/api/experimental/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/www/api/experimental/endpoints.py b/airflow/www/api/experimental/endpoints.py old mode 100644 new mode 100755 diff --git a/airflow/www/app.py b/airflow/www/app.py old mode 100644 new mode 100755 diff --git a/airflow/www/blueprints.py b/airflow/www/blueprints.py old mode 100644 new mode 100755 diff --git a/airflow/www/decorators.py b/airflow/www/decorators.py old mode 100644 new mode 100755 diff --git a/airflow/www/extensions/__init__.py b/airflow/www/extensions/__init__.py old mode 100644 new mode 100755 diff --git a/airflow/www/extensions/init_appbuilder.py b/airflow/www/extensions/init_appbuilder.py old mode 100644 new mode 100755 diff --git a/airflow/www/extensions/init_appbuilder_links.py b/airflow/www/extensions/init_appbuilder_links.py old mode 100644 new mode 100755 diff --git a/airflow/www/extensions/init_dagbag.py b/airflow/www/extensions/init_dagbag.py old mode 100644 new mode 100755 diff --git a/airflow/www/extensions/init_jinja_globals.py b/airflow/www/extensions/init_jinja_globals.py old mode 100644 new mode 100755 diff --git a/airflow/www/extensions/init_manifest_files.py b/airflow/www/extensions/init_manifest_files.py old mode 100644 new mode 100755 diff --git a/airflow/www/extensions/init_security.py b/airflow/www/extensions/init_security.py old mode 100644 new mode 100755 diff --git a/airflow/www/extensions/init_session.py b/airflow/www/extensions/init_session.py old mode 100644 new mode 100755 diff --git a/airflow/www/extensions/init_views.py b/airflow/www/extensions/init_views.py old mode 100644 new mode 100755 diff --git a/airflow/www/extensions/init_wsgi_middlewares.py b/airflow/www/extensions/init_wsgi_middlewares.py old mode 100644 new mode 100755 diff --git a/airflow/www/forms.py b/airflow/www/forms.py old mode 100644 new mode 100755 diff --git a/airflow/www/gunicorn_config.py b/airflow/www/gunicorn_config.py old mode 100644 new mode 100755 diff --git a/airflow/www/package.json b/airflow/www/package.json old mode 100644 new mode 100755 diff --git a/airflow/www/security.py b/airflow/www/security.py old mode 100644 new mode 100755 diff --git a/airflow/www/static/airflow.gif b/airflow/www/static/airflow.gif old mode 100644 new mode 100755 diff --git a/airflow/www/static/css/bootstrap-theme.css b/airflow/www/static/css/bootstrap-theme.css old mode 100644 new mode 100755 diff --git a/airflow/www/static/css/flash.css b/airflow/www/static/css/flash.css old mode 100644 new mode 100755 diff --git a/airflow/www/static/css/gantt.css b/airflow/www/static/css/gantt.css old mode 100644 new mode 100755 diff --git a/airflow/www/static/css/graph.css b/airflow/www/static/css/graph.css old mode 100644 new mode 100755 diff --git a/airflow/www/static/css/main.css b/airflow/www/static/css/main.css old mode 100644 new mode 100755 diff --git a/airflow/www/static/css/tree.css b/airflow/www/static/css/tree.css old mode 100644 new mode 100755 diff --git a/airflow/www/static/js/base.js b/airflow/www/static/js/base.js old mode 100644 new mode 100755 diff --git a/airflow/www/static/js/connection_form.js b/airflow/www/static/js/connection_form.js old mode 100644 new mode 100755 diff --git a/airflow/www/static/js/datetime-utils.js b/airflow/www/static/js/datetime-utils.js old mode 100644 new mode 100755 diff --git a/airflow/www/static/js/gantt-chart-d3v2.js b/airflow/www/static/js/gantt-chart-d3v2.js old mode 100644 new mode 100755 diff --git a/airflow/www/static/js/graph.js b/airflow/www/static/js/graph.js old mode 100644 new mode 100755 diff --git a/airflow/www/static/js/ie.js b/airflow/www/static/js/ie.js old mode 100644 new mode 100755 diff --git a/airflow/www/static/js/task-instances.js b/airflow/www/static/js/task-instances.js old mode 100644 new mode 100755 diff --git a/airflow/www/static/loading.gif b/airflow/www/static/loading.gif old mode 100644 new mode 100755 diff --git a/airflow/www/static/pin.svg b/airflow/www/static/pin.svg old mode 100644 new mode 100755 diff --git a/airflow/www/static/pin_100.png b/airflow/www/static/pin_100.png old mode 100644 new mode 100755 diff --git a/airflow/www/static/pin_25.png b/airflow/www/static/pin_25.png old mode 100644 new mode 100755 diff --git a/airflow/www/static/pin_32.png b/airflow/www/static/pin_32.png old mode 100644 new mode 100755 diff --git a/airflow/www/static/pin_35.png b/airflow/www/static/pin_35.png old mode 100644 new mode 100755 diff --git a/airflow/www/static/pin_40.png b/airflow/www/static/pin_40.png old mode 100644 new mode 100755 diff --git a/airflow/www/static/pin_large.png b/airflow/www/static/pin_large.png old mode 100644 new mode 100755 diff --git a/airflow/www/static/screenshots/gantt.png b/airflow/www/static/screenshots/gantt.png old mode 100644 new mode 100755 diff --git a/airflow/www/static/screenshots/graph.png b/airflow/www/static/screenshots/graph.png old mode 100644 new mode 100755 diff --git a/airflow/www/static/screenshots/tree.png b/airflow/www/static/screenshots/tree.png old mode 100644 new mode 100755 diff --git a/airflow/www/static/sort_asc.png b/airflow/www/static/sort_asc.png old mode 100644 new mode 100755 diff --git a/airflow/www/static/sort_both.png b/airflow/www/static/sort_both.png old mode 100644 new mode 100755 diff --git a/airflow/www/static/sort_desc.png b/airflow/www/static/sort_desc.png old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/chart.html b/airflow/www/templates/airflow/chart.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/circles.html b/airflow/www/templates/airflow/circles.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/code.html b/airflow/www/templates/airflow/code.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/config.html b/airflow/www/templates/airflow/config.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/confirm.html b/airflow/www/templates/airflow/confirm.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/conn_create.html b/airflow/www/templates/airflow/conn_create.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/conn_edit.html b/airflow/www/templates/airflow/conn_edit.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/dag.html b/airflow/www/templates/airflow/dag.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/dag_code.html b/airflow/www/templates/airflow/dag_code.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/dag_details.html b/airflow/www/templates/airflow/dag_details.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/dags.html b/airflow/www/templates/airflow/dags.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/duration_chart.html b/airflow/www/templates/airflow/duration_chart.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/gantt.html b/airflow/www/templates/airflow/gantt.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/graph.html b/airflow/www/templates/airflow/graph.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/master.html b/airflow/www/templates/airflow/master.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/model_list.html b/airflow/www/templates/airflow/model_list.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/noaccess.html b/airflow/www/templates/airflow/noaccess.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/redoc.html b/airflow/www/templates/airflow/redoc.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/task.html b/airflow/www/templates/airflow/task.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/task_instance.html b/airflow/www/templates/airflow/task_instance.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/ti_code.html b/airflow/www/templates/airflow/ti_code.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/ti_log.html b/airflow/www/templates/airflow/ti_log.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/traceback.html b/airflow/www/templates/airflow/traceback.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/tree.html b/airflow/www/templates/airflow/tree.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/trigger.html b/airflow/www/templates/airflow/trigger.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/variable_edit.html b/airflow/www/templates/airflow/variable_edit.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/variable_list.html b/airflow/www/templates/airflow/variable_list.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/version.html b/airflow/www/templates/airflow/version.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/airflow/xcom.html b/airflow/www/templates/airflow/xcom.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/analytics/google_analytics.html b/airflow/www/templates/analytics/google_analytics.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/analytics/metarouter.html b/airflow/www/templates/analytics/metarouter.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/analytics/segment.html b/airflow/www/templates/analytics/segment.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/appbuilder/flash.html b/airflow/www/templates/appbuilder/flash.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/appbuilder/index.html b/airflow/www/templates/appbuilder/index.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/appbuilder/navbar.html b/airflow/www/templates/appbuilder/navbar.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/appbuilder/navbar_menu.html b/airflow/www/templates/appbuilder/navbar_menu.html old mode 100644 new mode 100755 diff --git a/airflow/www/templates/appbuilder/navbar_right.html b/airflow/www/templates/appbuilder/navbar_right.html old mode 100644 new mode 100755 diff --git a/airflow/www/utils.py b/airflow/www/utils.py old mode 100644 new mode 100755 diff --git a/airflow/www/validators.py b/airflow/www/validators.py old mode 100644 new mode 100755 diff --git a/airflow/www/views.py b/airflow/www/views.py old mode 100644 new mode 100755 diff --git a/airflow/www/webpack.config.js b/airflow/www/webpack.config.js old mode 100644 new mode 100755 diff --git a/airflow/www/widgets.py b/airflow/www/widgets.py old mode 100644 new mode 100755 diff --git a/airflow/www/yarn.lock b/airflow/www/yarn.lock old mode 100644 new mode 100755 diff --git a/breeze-complete b/breeze-complete old mode 100644 new mode 100755 diff --git a/codecov.yml b/codecov.yml old mode 100644 new mode 100755 diff --git a/pylintrc b/pylintrc old mode 100644 new mode 100755 diff --git a/pytest.ini b/pytest.ini old mode 100644 new mode 100755 diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml old mode 100644 new mode 100755 index 6a521f9533de7..619156f5be775 --- a/scripts/ci/docker-compose/local.yml +++ b/scripts/ci/docker-compose/local.yml @@ -14,11 +14,12 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +--- version: "2.2" services: airflow: - stdin_open: true # docker run -i - tty: true # docker run -t + stdin_open: true # docker run -i + tty: true # docker run -t # We need to mount files an directories individually because some files # such apache_airflow.egg-info should not be mounted from host # we only mount those files that it makes sense to edit while developing @@ -55,16 +56,7 @@ services: - ../../../tests:/opt/airflow/tests:cached - ../../../kubernetes_tests:/opt/airflow/kubernetes_tests:cached - ../../../tmp:/tmp:cached + - ../../../metastore_browser:/opt/airflow/metastore_browser:cached # END automatically generated volumes from LOCAL_MOUNTS in _local_mounts.sh - - ../../../tmp:/opt/airflow/tmp:cached - # END automatically generated volumes from LOCAL_MOUNTS in - # _local_mounts.sh - environment: - - HOST_USER_ID - - HOST_GROUP_ID - - HOST_HOME=${HOME} - - HOST_AIRFLOW_SOURCES=${AIRFLOW_SOURCES} - - HOST_OS - - PYTHONDONTWRITEBYTECODE ports: - "${WEBSERVER_HOST_PORT}:8080" diff --git a/setup.cfg b/setup.cfg old mode 100644 new mode 100755 diff --git a/setup.py b/setup.py old mode 100644 new mode 100755 diff --git a/yamllint-config.yml b/yamllint-config.yml old mode 100644 new mode 100755 From e245bedcca94e35a30bd01fc5752890f5febd753 Mon Sep 17 00:00:00 2001 From: dan Date: Mon, 31 Aug 2020 07:40:10 -0400 Subject: [PATCH 161/162] squash! [AIRFLOW-5500] Fix the trigger_dag api in the case of nested subdags --- airflow/contrib/hooks/jira_hook.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 airflow/contrib/hooks/jira_hook.py diff --git a/airflow/contrib/hooks/jira_hook.py b/airflow/contrib/hooks/jira_hook.py old mode 100755 new mode 100644 From 15835e9ea106722dd19a97c6fe5d073c8614e4a4 Mon Sep 17 00:00:00 2001 From: dan Date: Mon, 31 Aug 2020 07:47:06 -0400 Subject: [PATCH 162/162] Trying to get this to work --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) mode change 100755 => 100644 README.md diff --git a/README.md b/README.md old mode 100755 new mode 100644 index 647b9f9ac4599..d2e60ae7d3195 --- a/README.md +++ b/README.md @@ -41,7 +41,7 @@ Use Airflow to author workflows as directed acyclic graphs (DAGs) of tasks. The -**Table of contents** + - [Requirements](#requirements) - [Getting started](#getting-started)