You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by je...@apache.org on 2021/12/15 18:57:58 UTC

[airflow] branch main updated: Deprecate smart sensors (#20151)

This is an automated email from the ASF dual-hosted git repository.

jedcunningham pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/main by this push:
     new 77813b4  Deprecate smart sensors (#20151)
77813b4 is described below

commit 77813b40db99683dcf14b557f9cddc50080c9a6a
Author: Jed Cunningham <66...@users.noreply.github.com>
AuthorDate: Wed Dec 15 11:57:25 2021 -0700

    Deprecate smart sensors (#20151)
    
    Smart sensors are being replaced with Deferrable Operators. As they were
    marked as an early-access feature, we can remove them before Airflow 3.
---
 UPDATING.md                                    |  5 +++++
 airflow/jobs/scheduler_job.py                  | 12 ++++++++++++
 airflow/sensors/base.py                        |  8 ++++++++
 docs/apache-airflow/concepts/deferring.rst     |  3 ++-
 docs/apache-airflow/concepts/smart-sensors.rst | 26 +++++++++++++++++---------
 5 files changed, 44 insertions(+), 10 deletions(-)

diff --git a/UPDATING.md b/UPDATING.md
index 1a20edd..c7d2625 100644
--- a/UPDATING.md
+++ b/UPDATING.md
@@ -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
 
diff --git a/airflow/jobs/scheduler_job.py b/airflow/jobs/scheduler_job.py
index 77aa33f..65e6274 100644
--- a/airflow/jobs/scheduler_job.py
+++ b/airflow/jobs/scheduler_job.py
@@ -48,6 +48,7 @@ from airflow.stats import Stats
 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
@@ -144,6 +145,17 @@ class SchedulerJob(BaseJob):
 
         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)
diff --git a/airflow/sensors/base.py b/airflow/sensors/base.py
index 0ca2ea3..b15596e 100644
--- a/airflow/sensors/base.py
+++ b/airflow/sensors/base.py
@@ -20,6 +20,7 @@ import datetime
 import functools
 import hashlib
 import time
+import warnings
 from datetime import timedelta
 from typing import Any, Callable, Dict, Iterable
 
@@ -41,6 +42,7 @@ from airflow.utils import timezone
 # 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)
@@ -177,6 +179,12 @@ class BaseSensorOperator(BaseOperator, SkipMixin):
         :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)
 
diff --git a/docs/apache-airflow/concepts/deferring.rst b/docs/apache-airflow/concepts/deferring.rst
index d9126c4..ca810d7 100644
--- a/docs/apache-airflow/concepts/deferring.rst
+++ b/docs/apache-airflow/concepts/deferring.rst
@@ -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
 ----------------------------
@@ -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.
diff --git a/docs/apache-airflow/concepts/smart-sensors.rst b/docs/apache-airflow/concepts/smart-sensors.rst
index fe84117..4fb9d3b 100644
--- a/docs/apache-airflow/concepts/smart-sensors.rst
+++ b/docs/apache-airflow/concepts/smart-sensors.rst
@@ -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.
@@ -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.