[AIRFLOW-6586] Improvements to gcs sensor (#7197)

* [AIRFLOW-6586] Improvements to gcs sensor

refactors GoogleCloudStorageUploadSessionCompleteSensor to use set instead of number of objects

add poke mode only decorator

assert that poke_mode_only applied to child of BaseSensorOperator

refactor tests

remove assert

[AIRFLOW-6586] Improvements to gcs sensor

refactors GoogleCloudStorageUploadSessionCompleteSensor to use set instead of number of objects

add poke mode only decorator

assert that poke_mode_only applied to child of BaseSensorOperator

remove assert

fix static checks

add back inadvertently remove requirements

pre-commit

fix typo

* gix gcs sensor unit test

* move poke_mode_only to base_sensor_operator module

* add sensor / poke_mode_only docs

* fix ci check add sensor how-to docs

* Update airflow/providers/google/cloud/sensors/gcs.py

Co-authored-by: Tomek Urbaszek <turbaszek@gmail.com>

* Update airflow/sensors/base_sensor_operator.py

Co-authored-by: Tomek Urbaszek <turbaszek@gmail.com>

* Update airflow/sensors/base_sensor_operator.py

Co-authored-by: Kamil Breguła <mik-laj@users.noreply.github.com>

* simplify class decorator

* remove type hint

* add note to UPDATING.md

* remove unecessary declaration of class member

* Fix to kwargs in UPDATING.md

Co-authored-by: Tomek Urbaszek <turbaszek@gmail.com>
Co-authored-by: Kamil Breguła <mik-laj@users.noreply.github.com>
This commit is contained in:
Jacob Ferriero 2020-05-19 14:14:28 -07:00 коммит произвёл GitHub
Родитель bae5cc2f5c
Коммит 499493c5c5
Не найден ключ, соответствующий данной подписи
Идентификатор ключа GPG: 4AEE18F83AFDEB23
6 изменённых файлов: 206 добавлений и 44 удалений

Просмотреть файл

@ -172,6 +172,29 @@ plugins =
- Added optional project_id argument to DataflowCreatePythonJobOperator
constructor.
### GCSUploadSessionCompleteSensor signature change
To provide more precise control in handling of changes to objects in
underlying GCS Bucket the constructor of this sensor now has changed.
- Old Behavior: This constructor used to optionally take ``previous_num_objects: int``.
- New replacement constructor kwarg: ``previous_objects: Optional[Set[str]]``.
Most users would not specify this argument because the bucket begins empty
and the user wants to treat any files as new.
Example of Updating usage of this sensor:
Users who used to call:
``GCSUploadSessionCompleteSensor(bucket='my_bucket', prefix='my_prefix', previous_num_objects=1)``
Will now call:
``GCSUploadSessionCompleteSensor(bucket='my_bucket', prefix='my_prefix', previous_num_objects={'.keep'})``
Where '.keep' is a single file at your prefix that the sensor should not consider new.
### Rename pool statsd metrics
Used slot has been renamed to running slot to make the name self-explanatory

Просмотреть файл

@ -21,11 +21,11 @@ This module contains Google Cloud Storage sensors.
import os
from datetime import datetime
from typing import Callable, List, Optional
from typing import Callable, List, Optional, Set
from airflow.exceptions import AirflowException
from airflow.providers.google.cloud.hooks.gcs import GCSHook
from airflow.sensors.base_sensor_operator import BaseSensorOperator
from airflow.sensors.base_sensor_operator import BaseSensorOperator, poke_mode_only
from airflow.utils.decorators import apply_defaults
@ -189,12 +189,14 @@ def get_time():
return datetime.now()
@poke_mode_only
class GCSUploadSessionCompleteSensor(BaseSensorOperator):
"""
Checks for changes in the number of objects at prefix in Google Cloud Storage
bucket and returns True if the inactivity period has passed with no
increase in the number of objects. Note, it is recommended to use reschedule
mode if you expect this sensor to run for hours.
increase in the number of objects. Note, this sensor will no behave correctly
in reschedule mode, as the state of the listed objects in the GCS bucket will
be lost between rescheduled invocations.
:param bucket: The Google Cloud Storage bucket where the objects are.
expected.
@ -209,8 +211,8 @@ class GCSUploadSessionCompleteSensor(BaseSensorOperator):
:param min_objects: The minimum number of objects needed for upload session
to be considered valid.
:type min_objects: int
:param previous_num_objects: The number of objects found during the last poke.
:type previous_num_objects: int
:param previous_objects: The set of object ids found during the last poke.
:type previous_objects: set[str]
:param allow_delete: Should this sensor consider objects being deleted
between pokes valid behavior. If true a warning message will be logged
when this happens. If false an error will be raised.
@ -233,7 +235,7 @@ class GCSUploadSessionCompleteSensor(BaseSensorOperator):
prefix: str,
inactivity_period: float = 60 * 60,
min_objects: int = 1,
previous_num_objects: int = 0,
previous_objects: Optional[Set[str]] = None,
allow_delete: bool = True,
google_cloud_conn_id: str = 'google_cloud_default',
delegate_to: Optional[str] = None,
@ -243,45 +245,56 @@ class GCSUploadSessionCompleteSensor(BaseSensorOperator):
self.bucket = bucket
self.prefix = prefix
if inactivity_period < 0:
raise ValueError("inactivity_period must be non-negative")
self.inactivity_period = inactivity_period
self.min_objects = min_objects
self.previous_num_objects = previous_num_objects
self.previous_objects = previous_objects if previous_objects else set()
self.inactivity_seconds = 0
self.allow_delete = allow_delete
self.google_cloud_conn_id = google_cloud_conn_id
self.delegate_to = delegate_to
self.last_activity_time = None
self.hook = None
def is_bucket_updated(self, current_num_objects: int) -> bool:
def _get_gcs_hook(self):
if not self.hook:
self.hook = GCSHook()
return self.hook
def is_bucket_updated(self, current_objects: Set[str]) -> bool:
"""
Checks whether new objects have been uploaded and the inactivity_period
has passed and updates the state of the sensor accordingly.
:param current_num_objects: number of objects in bucket during last poke.
:type current_num_objects: int
:param current_objects: set of object ids in bucket during last poke.
:type current_objects: set[str]
"""
if current_num_objects > self.previous_num_objects:
current_num_objects = len(current_objects)
if current_objects > self.previous_objects:
# When new objects arrived, reset the inactivity_seconds
# previous_num_objects for the next poke.
# and update previous_objects for the next poke.
self.log.info("New objects found at %s resetting last_activity_time.",
os.path.join(self.bucket, self.prefix))
self.log.debug("New objects: %s",
"\n".join(current_objects - self.previous_objects))
self.last_activity_time = get_time()
self.inactivity_seconds = 0
self.previous_num_objects = current_num_objects
self.previous_objects = current_objects
return False
if current_num_objects < self.previous_num_objects:
if self.previous_objects - current_objects:
# During the last poke interval objects were deleted.
if self.allow_delete:
self.previous_num_objects = current_num_objects
self.previous_objects = current_objects
self.last_activity_time = get_time()
self.log.warning(
"""
Objects were deleted during the last
poke interval. Updating the file counter and
resetting last_activity_time.
"""
%s
""", self.previous_objects - current_objects
)
return False
@ -314,5 +327,4 @@ class GCSUploadSessionCompleteSensor(BaseSensorOperator):
return False
def poke(self, context):
hook = GCSHook()
return self.is_bucket_updated(len(hook.list(self.bucket, prefix=self.prefix)))
return self.is_bucket_updated(set(self._get_gcs_hook().list(self.bucket, prefix=self.prefix)))

Просмотреть файл

@ -17,6 +17,7 @@
# under the License.
import hashlib
import os
from datetime import timedelta
from time import sleep
from typing import Any, Dict, Iterable
@ -176,3 +177,41 @@ class BaseSensorOperator(BaseOperator, SkipMixin):
if self.reschedule:
return BaseOperator.deps.fget(self) | {ReadyToRescheduleDep()}
return BaseOperator.deps.fget(self)
def poke_mode_only(cls):
"""
Class Decorator for child classes of BaseSensorOperator to indicate
that instances of this class are only safe to use poke mode.
Will decorate all methods in the class to assert they did not change
the mode from 'poke'.
:param cls: BaseSensor class to enforce methods only use 'poke' mode.
:type cls: type
"""
def decorate(cls_type):
def mode_getter(_):
return 'poke'
def mode_setter(_, value):
if value != 'poke':
raise ValueError(
f"cannot set mode to 'poke'.")
if not issubclass(cls_type, BaseSensorOperator):
raise ValueError(f"poke_mode_only decorator should only be "
f"applied to subclasses of BaseSensorOperator,"
f" got:{cls_type}.")
cls_type.mode = property(mode_getter, mode_setter)
return cls_type
return decorate(cls)
if 'BUILDING_AIRFLOW_DOCS' in os.environ:
# flake8: noqa: F811
# Monkey patch hook to get good function headers while building docs
apply_defaults = lambda x: x

Просмотреть файл

@ -206,3 +206,27 @@ Define an operator extra link
For your operator, you can :doc:`Define an extra link <define_extra_link>` that can
redirect users to external systems. For example, you can add a link that redirects
the user to the operator's manual.
Sensors
^^^^^^^^
Airflow provides a primitive for a special kind of operator, whose purpose is to
poll some state (e.g. presence of a file) on a regular interval until a
success criteria is met.
You can create any sensor your want by extending the :class:`airflow.sensors.base_sensor_operator.BaseSensorOperator`
defining a ``poke`` method to poll your external state and evaluate the success criteria.
Sensors have a powerful feature called ``'reschedule'`` mode which allows the sensor to
task to be rescheduled, rather than blocking a worker slot between pokes.
This is useful when you can tolerate a longer poll interval and expect to be
polling for a long time.
Reschedule mode comes with a caveat that your sensor cannot maintain internal state
between rescheduled executions. In this case you should decorate your sensor with
:meth:`airflow.sensors.base_sensor_operator.poke_mode_only`. This will let users know
that your sensor is not suitable for use with reschedule mode.
An example of a sensor that keeps internal state and cannot be used with reschedule mode
is :class:`airflow.providers.google.cloud.sensors.gcs.GCSUploadSessionCompleteSensor`.
It polls the number of objects at a prefix (this number is the internal state of the sensor)
and succeeds when there a certain amount of time has passed without the number of objects changing.

Просмотреть файл

@ -42,7 +42,7 @@ TEST_DAG_ID = 'unit_tests_gcs_sensor'
DEFAULT_DATE = datetime(2015, 1, 1)
MOCK_DATE_ARRAY = [datetime(2019, 2, 24, 12, 0, 0) - i * timedelta(seconds=10)
for i in range(20)]
for i in range(25)]
def next_time_side_effect():
@ -212,7 +212,6 @@ class TestGCSUploadSessionCompleteSensor(TestCase):
poke_interval=10,
min_objects=1,
allow_delete=False,
previous_num_objects=0,
dag=self.dag
)
@ -220,9 +219,9 @@ class TestGCSUploadSessionCompleteSensor(TestCase):
@mock.patch('airflow.providers.google.cloud.sensors.gcs.get_time', mock_time)
def test_files_deleted_between_pokes_throw_error(self):
self.sensor.is_bucket_updated(2)
self.sensor.is_bucket_updated({'a', 'b'})
with self.assertRaises(AirflowException):
self.sensor.is_bucket_updated(1)
self.sensor.is_bucket_updated({'a'})
@mock.patch('airflow.providers.google.cloud.sensors.gcs.get_time', mock_time)
def test_files_deleted_between_pokes_allow_delete(self):
@ -234,48 +233,49 @@ class TestGCSUploadSessionCompleteSensor(TestCase):
poke_interval=10,
min_objects=1,
allow_delete=True,
previous_num_objects=0,
dag=self.dag
)
self.sensor.is_bucket_updated(2)
self.sensor.is_bucket_updated({'a', 'b'})
self.assertEqual(self.sensor.inactivity_seconds, 0)
self.sensor.is_bucket_updated(1)
self.assertEqual(self.sensor.previous_num_objects, 1)
self.sensor.is_bucket_updated({'a'})
self.assertEqual(len(self.sensor.previous_objects), 1)
self.assertEqual(self.sensor.inactivity_seconds, 0)
self.sensor.is_bucket_updated(2)
self.sensor.is_bucket_updated({'a', 'c'})
self.assertEqual(self.sensor.inactivity_seconds, 0)
self.sensor.is_bucket_updated(2)
self.sensor.is_bucket_updated({'a', 'd'})
self.assertEqual(self.sensor.inactivity_seconds, 0)
self.sensor.is_bucket_updated({'a', 'd'})
self.assertEqual(self.sensor.inactivity_seconds, 10)
self.assertTrue(self.sensor.is_bucket_updated(2))
self.assertTrue(self.sensor.is_bucket_updated({'a', 'd'}))
@mock.patch('airflow.providers.google.cloud.sensors.gcs.get_time', mock_time)
def test_incoming_data(self):
self.sensor.is_bucket_updated(2)
self.sensor.is_bucket_updated({'a'})
self.assertEqual(self.sensor.inactivity_seconds, 0)
self.sensor.is_bucket_updated(3)
self.sensor.is_bucket_updated({'a', 'b'})
self.assertEqual(self.sensor.inactivity_seconds, 0)
self.sensor.is_bucket_updated(4)
self.sensor.is_bucket_updated({'a', 'b', 'c'})
self.assertEqual(self.sensor.inactivity_seconds, 0)
@mock.patch('airflow.providers.google.cloud.sensors.gcs.get_time', mock_time)
def test_no_new_data(self):
self.sensor.is_bucket_updated(2)
self.sensor.is_bucket_updated({'a'})
self.assertEqual(self.sensor.inactivity_seconds, 0)
self.sensor.is_bucket_updated(2)
self.sensor.is_bucket_updated({'a'})
self.assertEqual(self.sensor.inactivity_seconds, 10)
@mock.patch('airflow.providers.google.cloud.sensors.gcs.get_time', mock_time)
def test_no_new_data_success_criteria(self):
self.sensor.is_bucket_updated(2)
self.sensor.is_bucket_updated({'a'})
self.assertEqual(self.sensor.inactivity_seconds, 0)
self.sensor.is_bucket_updated(2)
self.sensor.is_bucket_updated({'a'})
self.assertEqual(self.sensor.inactivity_seconds, 10)
self.assertTrue(self.sensor.is_bucket_updated(2))
self.assertTrue(self.sensor.is_bucket_updated({'a'}))
@mock.patch('airflow.providers.google.cloud.sensors.gcs.get_time', mock_time)
def test_not_enough_objects(self):
self.sensor.is_bucket_updated(0)
self.sensor.is_bucket_updated(set())
self.assertEqual(self.sensor.inactivity_seconds, 0)
self.sensor.is_bucket_updated(0)
self.sensor.is_bucket_updated(set())
self.assertEqual(self.sensor.inactivity_seconds, 10)
self.assertFalse(self.sensor.is_bucket_updated(0))
self.assertFalse(self.sensor.is_bucket_updated(set()))

Просмотреть файл

@ -24,10 +24,10 @@ from unittest.mock import Mock, patch
from freezegun import freeze_time
from airflow.exceptions import AirflowException, AirflowRescheduleException, AirflowSensorTimeout
from airflow.models import DagRun, TaskInstance, TaskReschedule
from airflow.models import DagBag, DagRun, TaskInstance, TaskReschedule
from airflow.models.dag import DAG, settings
from airflow.operators.dummy_operator import DummyOperator
from airflow.sensors.base_sensor_operator import BaseSensorOperator
from airflow.sensors.base_sensor_operator import BaseSensorOperator, poke_mode_only
from airflow.ti_deps.deps.ready_to_reschedule import ReadyToRescheduleDep
from airflow.utils import timezone
from airflow.utils.state import State
@ -37,6 +37,7 @@ DEFAULT_DATE = datetime(2015, 1, 1)
TEST_DAG_ID = 'unit_test_dag'
DUMMY_OP = 'dummy_op'
SENSOR_OP = 'sensor_op'
DEV_NULL = 'dev/null'
class DummySensor(BaseSensorOperator):
@ -546,3 +547,66 @@ class TestBaseSensor(unittest.TestCase):
self.assertTrue(interval1 <= sensor.poke_interval)
self.assertTrue(interval2 >= sensor.poke_interval)
self.assertTrue(interval2 > interval1)
@poke_mode_only
class DummyPokeOnlySensor(BaseSensorOperator):
def __init__(self, poke_changes_mode=False, **kwargs):
self.mode = kwargs['mode']
super().__init__(**kwargs)
self.poke_changes_mode = poke_changes_mode
self.return_value = True
def poke(self, context):
if self.poke_changes_mode:
self.change_mode('reschedule')
return self.return_value
def change_mode(self, mode):
self.mode = mode
class TestPokeModeOnly(unittest.TestCase):
def setUp(self):
self.dagbag = DagBag(
dag_folder=DEV_NULL,
include_examples=True
)
self.args = {
'owner': 'airflow',
'start_date': DEFAULT_DATE
}
self.dag = DAG(TEST_DAG_ID, default_args=self.args)
def test_poke_mode_only_allows_poke_mode(self):
try:
sensor = DummyPokeOnlySensor(task_id='foo', mode='poke', poke_changes_mode=False,
dag=self.dag)
except ValueError:
self.fail("__init__ failed with mode='poke'.")
try:
sensor.poke({})
except ValueError:
self.fail("poke failed without changing mode from 'poke'.")
try:
sensor.change_mode('poke')
except ValueError:
self.fail("class method failed without changing mode from 'poke'.")
def test_poke_mode_only_bad_class_method(self):
sensor = DummyPokeOnlySensor(task_id='foo', mode='poke', poke_changes_mode=False,
dag=self.dag)
with self.assertRaises(ValueError):
sensor.change_mode('reschedule')
def test_poke_mode_only_bad_init(self):
with self.assertRaises(ValueError):
DummyPokeOnlySensor(task_id='foo', mode='reschedule',
poke_changes_mode=False, dag=self.dag)
def test_poke_mode_only_bad_poke(self):
sensor = DummyPokeOnlySensor(task_id='foo', mode='poke', poke_changes_mode=True,
dag=self.dag)
with self.assertRaises(ValueError):
sensor.poke({})