Skip to content

Commit

Permalink
Deprecate smart sensors (#20151)
Browse files Browse the repository at this point in the history
Smart sensors are being replaced with Deferrable Operators. As they were
marked as an early-access feature, we can remove them before Airflow 3.
  • Loading branch information
jedcunningham authored Dec 15, 2021
1 parent 705f129 commit 77813b4
Show file tree
Hide file tree
Showing 5 changed files with 44 additions and 10 deletions.
5 changes: 5 additions & 0 deletions UPDATING.md
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,11 @@ https://developers.google.com/style/inclusive-documentation

Continuing the effort to bind TaskInstance to a DagRun, XCom entries are now also tied to a DagRun. Use the ``run_id`` argument to specify the DagRun instead.

### Smart sensors deprecated

Smart sensors, an "early access" feature added in Airflow 2, are now deprecated and will be removed in Airflow 2.4.0. They have been superseded by Deferable Operators, added in Airflow 2.2.0.

See [Migrating to Deferrable Operators](https://airflow.apache.org/docs/apache-airflow/2.3.0/concepts/smart-sensors.html#migrating-to-deferrable-operators) for details on how to migrate.

## Airflow 2.2.2

Expand Down
12 changes: 12 additions & 0 deletions airflow/jobs/scheduler_job.py
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
from airflow.ti_deps.dependencies_states import EXECUTION_STATES
from airflow.utils import timezone
from airflow.utils.callback_requests import DagCallbackRequest, TaskCallbackRequest
from airflow.utils.docs import get_docs_url
from airflow.utils.event_scheduler import EventScheduler
from airflow.utils.retries import MAX_DB_RETRIES, retry_db_transaction, run_with_db_retries
from airflow.utils.session import create_session, provide_session
Expand Down Expand Up @@ -144,6 +145,17 @@ def __init__(

self.dagbag = DagBag(dag_folder=self.subdir, read_dags_from_db=True, load_op_links=False)

if conf.getboolean('smart_sensor', 'use_smart_sensor'):
compatible_sensors = set(
map(lambda l: l.strip(), conf.get('smart_sensor', 'sensors_enabled').split(','))
)
docs_url = get_docs_url('concepts/smart-sensors.html#migrating-to-deferrable-operators')
warnings.warn(
f'Smart sensors are deprecated, yet can be used for {compatible_sensors} sensors.'
f' Please use Deferrable Operators instead. See {docs_url} for more info.',
DeprecationWarning,
)

def register_signals(self) -> None:
"""Register signals that stop child processes"""
signal.signal(signal.SIGINT, self._exit_gracefully)
Expand Down
8 changes: 8 additions & 0 deletions airflow/sensors/base.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import functools
import hashlib
import time
import warnings
from datetime import timedelta
from typing import Any, Callable, Dict, Iterable

Expand All @@ -41,6 +42,7 @@
# Google Provider before 3.0.0 imported apply_defaults from here.
# See https://github.com/apache/airflow/issues/16035
from airflow.utils.decorators import apply_defaults # noqa: F401
from airflow.utils.docs import get_docs_url

# As documented in https://dev.mysql.com/doc/refman/5.7/en/datetime.html.
_MYSQL_TIMESTAMP_MAX = datetime.datetime(2038, 1, 19, 3, 14, 7, tzinfo=timezone.utc)
Expand Down Expand Up @@ -177,6 +179,12 @@ def register_in_sensor_service(self, ti, context):
:param context: TaskInstance template context from the ti.
:return: boolean
"""
docs_url = get_docs_url('concepts/smart-sensors.html#migrating-to-deferrable-operators')
warnings.warn(
'Your sensor is using Smart Sensors, which are deprecated.'
f' Please use Deferrable Operators instead. See {docs_url} for more info.',
DeprecationWarning,
)
poke_context = self.get_poke_context(context)
execution_context = self.get_execution_context(context)

Expand Down
3 changes: 2 additions & 1 deletion docs/apache-airflow/concepts/deferring.rst
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ That's it; everything else will be automatically handled for you. If you're upgr

Note that you cannot yet use the deferral ability from inside custom PythonOperator/TaskFlow Python functions; it is only available to traditional, class-based Operators at the moment.

.. _deferring/writing:

Writing Deferrable Operators
----------------------------
Expand Down Expand Up @@ -163,4 +164,4 @@ Note that every extra ``triggerer`` you run will result in an extra persistent c
Smart Sensors
-------------

Deferrable Operators essentially supersede :doc:`Smart Sensors <smart-sensors>`, and should be preferred for almost all situations. They do solve fundamentally the same problem; Smart Sensors, however, only work for certain Sensor workload styles, have no redundancy, and require a custom DAG to run at all times.
Deferrable Operators supersede :doc:`Smart Sensors <smart-sensors>`. They do solve fundamentally the same problem; Smart Sensors, however, only work for certain Sensor workload styles, have no redundancy, and require a custom DAG to run at all times.
26 changes: 17 additions & 9 deletions docs/apache-airflow/concepts/smart-sensors.rst
Original file line number Diff line number Diff line change
Expand Up @@ -23,15 +23,11 @@ Smart Sensors

.. warning::

This is an **early-access** feature and might change in incompatible ways in future Airflow versions.
However this feature can be considered bug-free, and Airbnb has been using this feature in production
since early 2020 and has significantly reduced their costs for heavy use of sensors.

.. note::

:doc:`Deferrable Operators <deferring>` are a more flexible way to achieve efficient long-running sensors,
as well as allowing Operators to also achieve similar efficiency gains. If you are considering writing a
new Smart Sensor, you may want to instead write it as a Deferrable Operator.
This is a **deprecated early-access** feature that will be removed in Airflow 2.4.0.
It is superseded by :doc:`Deferrable Operators <deferring>`, which offer a more flexible way to
achieve efficient long-running sensors, as well as allowing operators to also achieve similar
efficiency gains. If you are considering writing a new Smart Sensor, you should instead write it
as a Deferrable Operator.

The smart sensor is a service (run by a builtin DAG) which greatly reduces Airflow’s infrastructure
cost by consolidating multiple instances of small, light-weight Sensors into a single process.
Expand Down Expand Up @@ -96,3 +92,15 @@ Support new operators in the smart sensor service
include all key names used for initializing a sensor object.
* In ``airflow.cfg``, add the new operator's classname to ``[smart_sensor] sensors_enabled``.
All supported sensors' classname should be comma separated.

Migrating to Deferrable Operators
----------------------------------

There is not a direct migration path from Smart Sensors to :doc:`Deferrable Operators <deferring>`.
You have a few paths forward, depending on your needs and situation:

* Do nothing - your DAGs will continue to run as-is, however they will no longer get the optimization smart sensors brought
* Deferrable Operator - move to a Deferrable Operator that alleviates the need for a sensor all-together
* Deferrable Sensor - move to an async version of the sensor you are already using

See :ref:`Writing Deferrable Operators <deferring/writing>` for details on writing Deferrable Operators and Sensors.

0 comments on commit 77813b4

Please sign in to comment.