You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by po...@apache.org on 2023/01/11 13:37:56 UTC

[airflow] branch main updated: Add Public Interface description to Airflow documentation (#28300)

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

potiuk 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 352d492c66 Add Public Interface description to Airflow documentation (#28300)
352d492c66 is described below

commit 352d492c66e69e816fb1547e46fc1e3b7ba32066
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Wed Jan 11 14:37:39 2023 +0100

    Add Public Interface description to Airflow documentation (#28300)
    
    Airflow is a complex system and since it is a platform, it is supposed
    to be extended by the users by writing custom code wherever they miss
    functionality in Airflow's core or its providers.
    
    This page is an attempt to have a single place where we can express
    our intention of what is the Public Interface of Airflow that the
    user can depend on when implementing such customizations.
    
    This is never 100% possible, we know that some users workflows might
    depend on stuff that is internal implementation details or behaviours,
    however the user doing so should be aware of the risk they are taking
    by relying on something that was not intentionally exposed.
    
    This page is intended to serve as a guideline for the users who would
    like to make a decision to rely on some of the Airflow behaviours, so
    that they know whether the API they want to rely on were explicitly
    intended by Airflow community to expose as Public, or not.
    
    Co-authored-by: Bas Harenslak <ba...@users.noreply.github.com>
    
    Co-authored-by: Bas Harenslak <ba...@users.noreply.github.com>
---
 .../airflow_helmchart_bug_report.yml               |   2 +-
 airflow/decorators/__init__.pyi                    |   2 +-
 airflow/decorators/branch_python.py                |   2 +-
 airflow/decorators/sensor.py                       |   2 +
 airflow/triggers/base.py                           |   2 +-
 .../core-extensions/extra-links.rst                |   2 +-
 docs/apache-airflow-providers/index.rst            |   2 +-
 .../authoring-and-scheduling/plugins.rst           |   2 +-
 docs/apache-airflow/howto/custom-operator.rst      |   2 +-
 ...define_extra_link.rst => define-extra-link.rst} |   0
 docs/apache-airflow/howto/index.rst                |   2 +-
 docs/apache-airflow/howto/operator/index.rst       |   4 +-
 docs/apache-airflow/index.rst                      |   9 +-
 docs/apache-airflow/public-airflow-interface.rst   | 369 +++++++++++++++++++++
 docs/apache-airflow/python-api-ref.rst             | 175 ----------
 docs/apache-airflow/redirects.txt                  |   8 +-
 docs/conf.py                                       |   9 +-
 17 files changed, 400 insertions(+), 194 deletions(-)

diff --git a/.github/ISSUE_TEMPLATE/airflow_helmchart_bug_report.yml b/.github/ISSUE_TEMPLATE/airflow_helmchart_bug_report.yml
index 3111db5957..137fea97cf 100644
--- a/.github/ISSUE_TEMPLATE/airflow_helmchart_bug_report.yml
+++ b/.github/ISSUE_TEMPLATE/airflow_helmchart_bug_report.yml
@@ -63,7 +63,7 @@ body:
         you paste with ``` ```.
   - type: textarea
     attributes:
-      label: Docker Image customisations
+      label: Docker Image customizations
       description: What are the specific modification you've made in your image?
       placeholder: >
         Did you extend or customise the official Airflow image? Did you add any packages? Maybe
diff --git a/airflow/decorators/__init__.pyi b/airflow/decorators/__init__.pyi
index 0a6d534b24..b0edc7d2c2 100644
--- a/airflow/decorators/__init__.pyi
+++ b/airflow/decorators/__init__.pyi
@@ -166,7 +166,7 @@ class TaskDecoratorCollection:
     def branch(self, *, multiple_outputs: bool | None = None, **kwargs) -> TaskDecorator:
         """Create a decorator to wrap the decorated callable into a BranchPythonOperator.
 
-        For more information on how to use this decorator, see :ref:`howto/operator:BranchPythonOperator`.
+        For more information on how to use this decorator, see :ref:`concepts:branching`.
         Accepts arbitrary for operator kwarg. Can be reused in a single DAG.
 
         :param multiple_outputs: If set, function return value will be unrolled to multiple XCom values.
diff --git a/airflow/decorators/branch_python.py b/airflow/decorators/branch_python.py
index b7e3a94826..1c90917ed5 100644
--- a/airflow/decorators/branch_python.py
+++ b/airflow/decorators/branch_python.py
@@ -73,7 +73,7 @@ def branch_task(
     Wraps a python function into a BranchPythonOperator.
 
     For more information on how to use this operator, take a look at the guide:
-    :ref:`howto/operator:BranchPythonOperator`
+    :ref:`concepts:branching`
 
     Accepts kwargs for operator kwarg. Can be reused in a single DAG.
 
diff --git a/airflow/decorators/sensor.py b/airflow/decorators/sensor.py
index 2033968620..884249c107 100644
--- a/airflow/decorators/sensor.py
+++ b/airflow/decorators/sensor.py
@@ -28,6 +28,7 @@ from airflow.sensors.python import PythonSensor
 class DecoratedSensorOperator(PythonSensor):
     """
     Wraps a Python callable and captures args/kwargs when called for execution.
+
     :param python_callable: A reference to an object that is callable
     :param task_id: task Id
     :param op_args: a list of positional arguments that will get unpacked when
@@ -63,6 +64,7 @@ class DecoratedSensorOperator(PythonSensor):
 def sensor_task(python_callable: Callable | None = None, **kwargs) -> TaskDecorator:
     """
     Wraps a function into an Airflow operator.
+
     Accepts kwargs for operator kwarg. Can be reused in a single DAG.
     :param python_callable: Function to decorate
     """
diff --git a/airflow/triggers/base.py b/airflow/triggers/base.py
index 6bfc0883ef..06bce36a4c 100644
--- a/airflow/triggers/base.py
+++ b/airflow/triggers/base.py
@@ -32,7 +32,7 @@ class BaseTrigger(abc.ABC, LoggingMixin):
      - Actively running in a trigger worker
 
     We use the same class for both situations, and rely on all Trigger classes
-    to be able to return the (Airflow-JSON-encodable) arguments that will
+    to be able to return the arguments (possible to encode with Airflow-JSON) that will
     let them be re-instantiated elsewhere.
     """
 
diff --git a/docs/apache-airflow-providers/core-extensions/extra-links.rst b/docs/apache-airflow-providers/core-extensions/extra-links.rst
index 884f5906b2..5fe8421b1d 100644
--- a/docs/apache-airflow-providers/core-extensions/extra-links.rst
+++ b/docs/apache-airflow-providers/core-extensions/extra-links.rst
@@ -26,7 +26,7 @@ its own extra links that can redirect users to external systems. The extra link
 will be available on the task page.
 
 The operator extra links are explained in
-:doc:`apache-airflow:howto/define_extra_link` and here you can also see the extra links
+:doc:`apache-airflow:howto/define-extra-link` and here you can also see the extra links
 provided by the community-managed providers:
 
 .. airflow-extra-links::
diff --git a/docs/apache-airflow-providers/index.rst b/docs/apache-airflow-providers/index.rst
index e81e03b046..86bf050b81 100644
--- a/docs/apache-airflow-providers/index.rst
+++ b/docs/apache-airflow-providers/index.rst
@@ -205,7 +205,7 @@ Displaying package information in CLI/API:
 Exposing customized functionality to the Airflow's core:
 
 * ``extra-links`` - this field should contain the list of all operator class names that are adding extra links
-  capability. See :doc:`apache-airflow:howto/define_extra_link` for description of how to add extra link
+  capability. See :doc:`apache-airflow:howto/define-extra-link` for description of how to add extra link
   capability to the operators of yours.
 
 * ``connection-types`` - this field should contain the list of all connection types together with hook
diff --git a/docs/apache-airflow/authoring-and-scheduling/plugins.rst b/docs/apache-airflow/authoring-and-scheduling/plugins.rst
index bd74de93d0..7c1ebb03b0 100644
--- a/docs/apache-airflow/authoring-and-scheduling/plugins.rst
+++ b/docs/apache-airflow/authoring-and-scheduling/plugins.rst
@@ -257,7 +257,7 @@ definitions in Airflow.
         appbuilder_views = [v_appbuilder_package, v_appbuilder_nomenu_package]
         appbuilder_menu_items = [appbuilder_mitem, appbuilder_mitem_toplevel]
 
-.. seealso:: :doc:`/howto/define_extra_link`
+.. seealso:: :doc:`/howto/define-extra-link`
 
 Exclude views from CSRF protection
 ----------------------------------
diff --git a/docs/apache-airflow/howto/custom-operator.rst b/docs/apache-airflow/howto/custom-operator.rst
index 15f1e79e2c..ea49db33de 100644
--- a/docs/apache-airflow/howto/custom-operator.rst
+++ b/docs/apache-airflow/howto/custom-operator.rst
@@ -270,7 +270,7 @@ If you use a non-existing lexer then the value of the template field will be ren
 Define an operator extra link
 ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
 
-For your operator, you can :doc:`Define an extra link <define_extra_link>` that can
+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.
 
diff --git a/docs/apache-airflow/howto/define_extra_link.rst b/docs/apache-airflow/howto/define-extra-link.rst
similarity index 100%
rename from docs/apache-airflow/howto/define_extra_link.rst
rename to docs/apache-airflow/howto/define-extra-link.rst
diff --git a/docs/apache-airflow/howto/index.rst b/docs/apache-airflow/howto/index.rst
index 5d6b54fd77..d6e1e55e4f 100644
--- a/docs/apache-airflow/howto/index.rst
+++ b/docs/apache-airflow/howto/index.rst
@@ -46,7 +46,7 @@ configuring an Airflow environment.
     run-behind-proxy
     run-with-systemd
     use-test-config
-    define_extra_link
+    define-extra-link
     email-config
     dynamic-dag-generation
     docker-compose/index
diff --git a/docs/apache-airflow/howto/operator/index.rst b/docs/apache-airflow/howto/operator/index.rst
index e734e81b0d..2ca7a5e965 100644
--- a/docs/apache-airflow/howto/operator/index.rst
+++ b/docs/apache-airflow/howto/operator/index.rst
@@ -24,9 +24,7 @@ An operator represents a single, ideally idempotent, task. Operators
 determine what actually executes when your DAG runs.
 
 .. note::
-    See the :doc:`Operators Concepts </core-concepts/operators>` documentation and the
-    :doc:`Operators API Reference </python-api-ref>` for more
-    information.
+    See the :doc:`Operators Concepts </core-concepts/operators>` documentation.
 
 .. toctree::
     :maxdepth: 2
diff --git a/docs/apache-airflow/index.rst b/docs/apache-airflow/index.rst
index 887479422b..eac8ef6cde 100644
--- a/docs/apache-airflow/index.rst
+++ b/docs/apache-airflow/index.rst
@@ -79,6 +79,7 @@ seen running over time:
 Each column represents one DAG run. These are two of the most used views in Airflow, but there are several
 other views which allow you to deep dive into the state of your workflows.
 
+
 Why Airflow?
 =========================================
 Airflow is a batch workflow orchestration platform. The Airflow framework contains operators to connect with
@@ -107,8 +108,12 @@ The open-source nature of Airflow ensures you work on components developed, test
 blogs posts, articles, conferences, books, and more. You can connect with other peers via several channels
 such as `Slack <https://s.apache.org/airflow-slack>`_ and mailing lists.
 
+Airflow as a Platform is highly customizable. By utilizing :doc:`public-airflow-interface` you can extend
+and customize almost every aspect of Airflow.
+
 Why not Airflow?
-=========================================
+================
+
 Airflow was built for finite batch workflows. While the CLI and REST API do allow triggering workflows,
 Airflow was not built for infinitely-running event-based workflows. Airflow is not a streaming solution.
 However, a streaming system such as Apache Kafka is often seen working together with Apache Airflow. Kafka can
@@ -137,6 +142,7 @@ so coding will always be required.
     authoring-and-scheduling/index
     administration-and-deployment/index
     integration
+    public-airflow-interface
     best-practices
     faq
     Release Policies <release-process>
@@ -150,7 +156,6 @@ so coding will always be required.
     Operators and hooks <operators-and-hooks-ref>
     CLI <cli-and-env-variables-ref>
     Templates <templates-ref>
-    Python API <python-api-ref>
     Stable REST API <stable-rest-api-ref>
     deprecated-rest-api-ref
     Configurations <configurations-ref>
diff --git a/docs/apache-airflow/public-airflow-interface.rst b/docs/apache-airflow/public-airflow-interface.rst
new file mode 100644
index 0000000000..523e6aa215
--- /dev/null
+++ b/docs/apache-airflow/public-airflow-interface.rst
@@ -0,0 +1,369 @@
+ .. 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.
+
+Public Interface of Airflow
+...........................
+
+The Public Interface of Apache Airflow is a set of interfaces that allow developers to interact
+with and access certain features of the Apache Airflow system. This includes operations such as
+creating and managing DAGs (Directed Acyclic Graphs), managing tasks and their dependencies,
+and extending Airflow capabilities by writing new executors, plugins, operators and providers. The
+Public Interface can be useful for building custom tools and integrations with other systems,
+and for automating certain aspects of the Airflow workflow.
+
+Using Airflow Public Interfaces
+===============================
+
+Using Airflow Public Interfaces is needed when you want to interact with Airflow programmatically:
+
+* When you are extending Airflow classes such as Operators and Hooks. This can be done by DAG authors to add missing functionality in their DAGs or by those who write reusable custom operators for other DAG authors.
+* When writing new :doc:`Plugins <authoring-and-scheduling/plugins>` that extend Airflow's functionality beyond
+  DAG building blocks. Secrets, Timetables, Triggers, Listeners are all examples of such functionality. This
+  is usually done by users who manage Airflow instances.
+* Bundling custom Operators, Hooks, Plugins and releasing them together via
+  :doc:`provider packages <apache-airflow-providers:index>` - this is usually done by those who intend to
+  provide a reusable set of functionality for external services or applications Airflow integrates with.
+
+All the ways above involve extending or using Airflow Python classes and functions. The classes
+and functions mentioned below can be relied on to keep backwards-compatible signatures and behaviours within
+MAJOR version of Airflow. On the other hand, classes and methods starting with ``_`` (also known
+as protected Python methods) and ``__`` (also known as private Python methods) are not part of the Public
+Airflow Interface and might change at any time.
+
+You can also use Airflow's Public Interface via the `Stable REST API <stable-rest-api-ref>`_ (based on the
+OpenAPI specification). For specific needs you can also use the
+`Airflow Command Line Interface (CLI) <cli-and-env-variables-ref.rst>`_ though it's behaviour might change
+in details (such as output format and available flags) so if you want to rely on those in programmatic
+way, the Stable REST API is recommended.
+
+
+Using the Public Interface for DAG Authors
+==========================================
+
+DAGs
+----
+
+The DAG is Airflow's core entity that represents a recurring workflow. You can create a DAG by
+instantiating the :class:`~airflow.models.dag.DAG` class in your DAG file.
+
+Airflow has a set of example DAGs that you can use to learn how to write DAGs
+
+.. toctree::
+  :includehidden:
+  :glob:
+  :maxdepth: 1
+
+  _api/airflow/example_dags/index
+
+You can read more about DAGs in :doc:`DAGs <core-concepts/dags>`.
+
+.. _pythonapi:operators:
+
+Operators
+---------
+
+Operators allow for generation of certain types of tasks that become nodes in
+the DAG when instantiated.
+
+There are 3 main types of operators:
+
+- Operators that performs an **action**, or tell another system to
+  perform an action
+- **Transfer** operators move data from one system to another
+- **Sensors** are a certain type of operator that will keep running until a
+  certain criterion is met. Examples include a specific file landing in HDFS or
+  S3, a partition appearing in Hive, or a specific time of the day. Sensors
+  are derived from :class:`~airflow.sensors.base.BaseSensorOperator` and run a poke
+  method at a specified :attr:`~airflow.sensors.base.BaseSensorOperator.poke_interval` until it
+  returns ``True``.
+
+All operators are derived from :class:`~airflow.models.baseoperator.BaseOperator` and acquire much
+functionality through inheritance. Since this is the core of the engine,
+it's worth taking the time to understand the parameters of :class:`~airflow.models.baseoperator.BaseOperator`
+to understand the primitive features that can be leveraged in your DAGs.
+
+Airflow has a set of Operators that are considered public. You are also free to extend their functionality
+by extending them:
+
+.. toctree::
+  :includehidden:
+  :glob:
+  :maxdepth: 1
+
+  _api/airflow/operators/index
+
+  _api/airflow/sensors/index
+
+
+You can read more about the operators in :doc:`core-concepts/operators`, :doc:`core-concepts/sensors`.
+Also you can learn how to write a custom operator in :doc:`howto/custom-operator`.
+
+.. _pythonapi:hooks:
+
+Hooks
+-----
+
+Hooks are interfaces to external platforms and databases, implementing a common
+interface when possible and acting as building blocks for operators. All hooks
+are derived from :class:`~airflow.hooks.base.BaseHook`.
+
+Airflow has a set of Hooks that are considered public. You are free to extend their functionality
+by extending them:
+
+.. toctree::
+  :includehidden:
+  :glob:
+  :maxdepth: 1
+
+  _api/airflow/hooks/index
+
+Public Airflow utilities
+------------------------
+
+When writing or extending Hooks and Operators, DAG authors and developers can
+use the following classes:
+
+* The :class:`~airflow.models.connection.Connection`, which provides access to external service credentials and configuration.
+* The :class:`~airflow.models.variable.Variable`, which provides access to Airflow configuration variables.
+* The :class:`~airflow.models.xcom.XCom` which are used to access to inter-task communication data.
+
+You can read more about the public Airflow utilities in :doc:`howto/connection`,
+:doc:`core-concepts/variables`, :doc:`core-concepts/xcoms`
+
+Public Exceptions
+-----------------
+
+When writing the custom Operators and Hooks, you can handle and raise public Exceptions that Airflow
+exposes:
+
+.. toctree::
+  :includehidden:
+  :glob:
+  :maxdepth: 1
+
+  _api/airflow/exceptions/index
+
+
+Using Public Interface to extend Airflow capabilities
+=====================================================
+
+Airflow uses Plugin mechanism to extend Airflow platform capabilities. They allow to extend
+Airflow UI but also they are the way to expose the below customizations (Triggers, Timetables, Listeners, etc.).
+Providers can also implement plugin endpoints and customize Airflow UI and the customizations.
+
+You can read more about plugins in :doc:`authoring-and-scheduling/plugins`. You can read how to extend
+Airflow UI in :doc:`howto/custom-view-plugin`. Note that there are some simple customizations of the UI
+that do not require plugins - you can read more about them in :doc:`howto/customize-ui`.
+
+Here are the ways how Plugins can be used to extend Airflow:
+
+Triggers
+--------
+
+Airflow uses Triggers to implement ``asyncio`` compatible Deferrable Operators.
+All Triggers derive from :class:`~airflow.triggers.base.BaseTrigger`.
+
+Airflow has a set of Triggers that are considered public. You are free to extend their functionality
+by extending them:
+
+.. toctree::
+  :includehidden:
+  :glob:
+  :maxdepth: 1
+
+  _api/airflow/triggers/index
+
+You can read more about Triggers in :doc:`authoring-and-scheduling/deferring`.
+
+Timetables
+----------
+
+Custom timetable implementations provide Airflow's scheduler additional logic to
+schedule DAG runs in ways not possible with built-in schedule expressions.
+All Timetables derive from :class:`~airflow.timetables.base.Timetable`.
+
+Airflow has a set of Timetables that are considered public. You are free to extend their functionality
+by extending them:
+
+.. toctree::
+  :includehidden:
+  :maxdepth: 1
+
+  _api/airflow/timetables/index
+
+You can read more about Timetables in :doc:`howto/timetable`.
+
+Listeners
+---------
+
+Listeners enable you to respond to DAG/Task lifecycle events.
+
+This is implemented via :class:`~airflow.listeners.listener.ListenerManager` class that provides hooks that
+can be implemented to respond to DAG/Task lifecycle events.
+
+.. versionadded:: 2.5
+
+   Listener public interface has been added in version 2.5.
+
+You can read more about Listeners in :doc:`administration-and-deployment/listeners`.
+
+Extra Links
+-----------
+
+Extra links are dynamic links that could be added to Airflow independently from custom Operators. Normally
+they can be defined by the Operators, but plugins allow you to override the links on a global level.
+
+You can read more about the Extra Links in :doc:`/howto/define-extra-link`.
+
+Using Public Interface to integrate with external services and applications
+===========================================================================
+
+
+Tasks in Airflow can orchestrate external services via Hooks and Operators. The core functionality of
+Airflow (such as authentication) can also be extended to leverage external services.
+You can read more about providers :doc:`provider packages <apache-airflow-providers:index>` and core
+extensions they can provide in :doc:`provider packages <apache-airflow-providers:core-extensions/index>`.
+
+Executors
+---------
+
+Executors are the mechanism by which task instances get run. All executors are
+derived from :class:`~airflow.executors.base_executor.BaseExecutor`. There are several
+executor implementations built-in Airflow, each with its own unique characteristics and capabilities.
+
+Airflow has a set of Executors that are considered public. You are free to extend their functionality
+by extending them:
+
+.. toctree::
+  :includehidden:
+  :glob:
+  :maxdepth: 1
+
+  _api/airflow/executors/index
+
+You can read more about executors in :doc:`core-concepts/executor/index`.
+
+.. versionadded:: 2.6
+
+  Executor interface was available in earlier version of Airflow but only as of version 2.6 executors are
+  fully decoupled and Airflow does not rely on built-in set of executors.
+  You could have implemented (and succeeded) with implementing Executors before Airflow 2.6 and a number
+  of people succeeded in doing so, but there were some hard-coded behaviours that preferred in-built
+  executors, and custom executors could not provide full functionality that built-in executors had.
+
+Secrets Backends
+----------------
+
+Airflow can be configured to rely on secrets backends to retrieve
+:class:`~airflow.models.connection.Connection` and :class:`~airflow.models.Variables`.
+All secrets backends derive from :class:`~airflow.secrets.BaseSecretsBackend`.
+
+All Secrets Backend implementations are public. You can extend their functionality:
+
+.. toctree::
+  :includehidden:
+  :glob:
+  :maxdepth: 1
+
+  _api/airflow/secrets/index
+
+You can read more about Secret Backends in :doc:`administration-and-deployment/security/secrets/secrets-backend/index`.
+You can also find all the available Secrets Backends implemented in community providers
+in :doc:`apache-airflow-providers:core-extensions/secrets-backends`.
+
+Authentication Backends
+-----------------------
+
+Authentication backends can extend the way how Airflow authentication mechanism works. You can find out more
+about authentication in :doc:`apache-airflow-providers:core-extensions/auth-backends` that also shows available
+Authentication backends implemented in the community providers.
+
+Connections
+-----------
+
+When creating Hooks, you can add custom Connections. You can read more
+about connections in :doc:`apache-airflow-providers:core-extensions/connections` for available
+Connections implemented in the community providers.
+
+Extra Links
+-----------
+
+When creating Hooks, you can add custom Extra Links that are displayed when the tasks are run.
+You can find out more about extra links in :doc:`apache-airflow-providers:core-extensions/extra-links`
+that also shows available extra links implemented in the community providers.
+
+Logging and Monitoring
+----------------------
+
+You can extend the way how logs are written by Airflow. You can find out more about log writing in
+:doc:`administration-and-deployment/logging-monitoring/index`.
+
+The :doc:`apache-airflow-providers:core-extensions/logging` that also shows available log writers
+implemented in the community providers.
+
+Decorators
+----------
+DAG authors can use decorators to author DAGs using the :doc:`TaskFlow <core-concepts/taskflow>` concept.
+All Decorators derive from :class:`~airflow.decorators.base.TaskDecorator`.
+
+Airflow has a set of Decorators that are considered public. You are free to extend their functionality
+by extending them:
+
+.. toctree::
+  :includehidden:
+  :maxdepth: 1
+
+  _api/airflow/decorators/index
+
+You can read more about creating custom Decorators in :doc:`howto/create-custom-decorator`.
+
+Email notifications
+-------------------
+
+Airflow has a built-in way of sending email notifications and it allows to extend it by adding custom
+email notification classes. You can read more about email notifications in :doc:`howto/email-config`.
+
+Cluster Policies
+----------------
+
+Cluster Policies are the way to dynamically apply cluster-wide policies to the DAGs being parsed or tasks
+being executed. You can read more about Cluster Policies in :doc:`administration-and-deployment/cluster-policies`.
+
+Lineage
+-------
+
+Airflow can help track origins of data, what happens to it and where it moves over time. You can read more
+about lineage in :doc:`administration-and-deployment/lineage`.
+
+
+What is not part of the Public Interface of Apache Airflow?
+===========================================================
+
+Everything not mentioned in this document should be considered as non-Public Interface.
+
+Sometimes in other applications those components could be relied on to keep backwards compatibility,
+but in Airflow they are not parts of the Public Interface and might change any time:
+
+* `Database structure <database-erd-ref>`_ is considered to be an internal implementation
+  detail and you should not assume the structure is going to be maintained in a
+  backwards-compatible way.
+
+* `Web UI <ui>`_ is continuously evolving and there are no backwards compatibility guarantees on HTML elements.
+
+* Python classes except those explicitly mentioned in this document, are considered an
+  internal implementation detail and you should not assume they will be maintained
+  in a backwards-compatible way.
diff --git a/docs/apache-airflow/python-api-ref.rst b/docs/apache-airflow/python-api-ref.rst
deleted file mode 100644
index 9e385f846d..0000000000
--- a/docs/apache-airflow/python-api-ref.rst
+++ /dev/null
@@ -1,175 +0,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
-
- ..   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.
-
-
-
-Python API Reference
-====================
-
-.. _pythonapi:dags:
-
-DAGs
----------
-The DAG is Airflow's core model that represents a recurring workflow. Check out :class:`~airflow.models.dag.DAG` for details.
-
-.. _pythonapi:operators:
-
-Operators
----------
-Operators allow for generation of certain types of tasks that become nodes in
-the DAG when instantiated. All operators derive from :class:`~airflow.models.baseoperator.BaseOperator` and
-inherit many attributes and methods that way.
-
-There are 3 main types of operators:
-
-- Operators that performs an **action**, or tell another system to
-  perform an action
-- **Transfer** operators move data from one system to another
-- **Sensors** are a certain type of operator that will keep running until a
-  certain criterion is met. Examples include a specific file landing in HDFS or
-  S3, a partition appearing in Hive, or a specific time of the day. Sensors
-  are derived from :class:`~airflow.sensors.base.BaseSensorOperator` and run a poke
-  method at a specified :attr:`~airflow.sensors.base.BaseSensorOperator.poke_interval` until it returns ``True``.
-
-BaseOperator
-''''''''''''
-All operators are derived from :class:`~airflow.models.baseoperator.BaseOperator` and acquire much
-functionality through inheritance. Since this is the core of the engine,
-it's worth taking the time to understand the parameters of :class:`~airflow.models.baseoperator.BaseOperator`
-to understand the primitive features that can be leveraged in your
-DAGs.
-
-BaseSensorOperator
-''''''''''''''''''
-All sensors are derived from :class:`~airflow.sensors.base.BaseSensorOperator`. All sensors inherit
-the :attr:`~airflow.sensors.base.BaseSensorOperator.timeout` and :attr:`~airflow.sensors.base.BaseSensorOperator.poke_interval` on top of the :class:`~airflow.models.baseoperator.BaseOperator`
-attributes.
-
-Operators packages
-''''''''''''''''''
-All operators are in the following packages:
-
-.. toctree::
-  :includehidden:
-  :glob:
-  :maxdepth: 1
-
-  _api/airflow/operators/index
-
-  _api/airflow/sensors/index
-
-
-.. _pythonapi:hooks:
-
-Hooks
------
-Hooks are interfaces to external platforms and databases, implementing a common
-interface when possible and acting as building blocks for operators. All hooks
-are derived from :class:`~airflow.hooks.base.BaseHook`.
-
-Hooks packages
-''''''''''''''
-All hooks are in the following packages:
-
-.. toctree::
-  :includehidden:
-  :glob:
-  :maxdepth: 1
-
-  _api/airflow/hooks/index
-
-Executors
----------
-Executors are the mechanism by which task instances get run. All executors are
-derived from :class:`~airflow.executors.base_executor.BaseExecutor`.
-
-Executors packages
-''''''''''''''''''
-All executors are in the following packages:
-
-.. toctree::
-  :includehidden:
-  :glob:
-  :maxdepth: 1
-
-  _api/airflow/executors/index
-
-Models
-------
-Models are built on top of the SQLAlchemy ORM Base class, and instances are
-persisted in the database.
-
-.. toctree::
-  :includehidden:
-  :glob:
-  :maxdepth: 1
-
-  _api/airflow/models/index
-
-.. _pythonapi:exceptions:
-
-Exceptions
-----------
-
-.. toctree::
-  :includehidden:
-  :glob:
-  :maxdepth: 1
-
-  _api/airflow/exceptions/index
-
-Secrets Backends
-----------------
-Airflow relies on secrets backends to retrieve :class:`~airflow.models.connection.Connection` objects.
-All secrets backends derive from :class:`~airflow.secrets.BaseSecretsBackend`.
-
-.. toctree::
-  :includehidden:
-  :glob:
-  :maxdepth: 1
-
-  _api/airflow/secrets/index
-
-Timetables
-----------
-Custom timetable implementations provide Airflow's scheduler additional logic to
-schedule DAG runs in ways not possible with built-in schedule expressions.
-
-.. toctree::
-  :includehidden:
-  :maxdepth: 1
-
-  _api/airflow/timetables/index
-
-Example DAGs
-------------
-
-.. toctree::
-  :includehidden:
-  :maxdepth: 1
-
-  _api/airflow/example_dags/index
-
-Utils
------
-
-.. toctree::
-  :hidden:
-  :glob:
-  :maxdepth: 1
-
-  _api/airflow/utils/dag_parsing_context/index
diff --git a/docs/apache-airflow/redirects.txt b/docs/apache-airflow/redirects.txt
index 20e11a493a..b99a5b253c 100644
--- a/docs/apache-airflow/redirects.txt
+++ b/docs/apache-airflow/redirects.txt
@@ -46,7 +46,7 @@ start/index.rst start.rst
 
 # References
 cli-ref.rst cli-and-env-variables-ref.rst
-_api/index.rst python-api-ref.rst
+_api/index.rst public-airflow-interface.rst
 rest-api-ref.rst deprecated-rest-api-ref.rst
 macros-ref.rst templates-ref.rst
 
@@ -130,3 +130,9 @@ executor/celery.rst core-concepts/executor/celery.rst
 executor/local.rst core-concepts/executor/local.rst
 executor/sequential.rst core-concepts/executor/sequential.rst
 upgrading-from-1-10/upgrade-check.rst howto/upgrading-from-1-10/upgrade-check.rst
+
+# Python API
+python-api-ref.rst public-airflow-interface.rst
+
+# Typos
+howto/define_extra_link.rst howto/define-extra-link.rst
diff --git a/docs/conf.py b/docs/conf.py
index 32a898a027..f6c660141c 100644
--- a/docs/conf.py
+++ b/docs/conf.py
@@ -221,17 +221,18 @@ if PACKAGE_NAME == "apache-airflow":
 
     browsable_packages = {
         "hooks",
+        "decorators",
         "example_dags",
         "executors",
-        "models",
         "operators",
         "providers",
         "secrets",
         "sensors",
         "timetables",
+        "triggers",
         "utils",
     }
-    browseable_utils = {"dag_parsing_context.py"}
+    browsable_utils: set[str] = set()
 
     root = ROOT_DIR / "airflow"
     for path in root.iterdir():
@@ -240,9 +241,9 @@ if PACKAGE_NAME == "apache-airflow":
         if path.is_dir() and path.name not in browsable_packages:
             exclude_patterns.append(f"_api/airflow/{path.name}")
 
-    # Don't include all of utils, just the specific ones we include in python-api-ref
+    # Don't include all of utils, just the specific ones we decoded to include
     for path in (root / "utils").iterdir():
-        if path.name not in browseable_utils:
+        if path.name not in browsable_utils:
             exclude_patterns.append(_get_rst_filepath_from_path(path))
 elif PACKAGE_NAME != "docker-stack":
     exclude_patterns.extend(