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 2022/06/29 14:27:04 UTC

[airflow] branch move-setup-dependencies-to-providers updated (625d131441 -> 6349b5e578)

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

potiuk pushed a change to branch move-setup-dependencies-to-providers
in repository https://gitbox.apache.org/repos/asf/airflow.git


    omit 625d131441 Move provider dependencies to inside provider folders
     add 13908c2c91 Adding generic `SqlToSlackOperator` (#24663)
     add cef97fccd5 fix connection extra parameter `auth_mechanism` in `HiveMetastoreHook` and `HiveServer2Hook` (#24713)
     new 6349b5e578 Move provider dependencies to inside provider folders

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (625d131441)
            \
             N -- N -- N   refs/heads/move-setup-dependencies-to-providers (6349b5e578)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .pre-commit-config.yaml                            |   2 +-
 airflow/provider.yaml.schema.json                  |   2 +-
 airflow/providers/apache/hive/hooks/hive.py        |  31 ++-
 .../providers/presto/transfers/presto_to_slack.py  |  87 ++-------
 airflow/providers/slack/provider.yaml              |   6 +
 .../slack/transfers}/__init__.py                   |   0
 .../transfers/sql_to_slack.py}                     | 122 ++++++++----
 .../snowflake/transfers/snowflake_to_slack.py      | 103 +++-------
 docs/apache-airflow-providers-slack/index.rst      |  10 +
 .../operators/sql_to_slack.rst}                    |  18 +-
 .../pre_commit_check_2_2_compatibility.py          |   5 +-
 tests/providers/apache/hive/hooks/test_hive.py     |   2 +-
 .../presto/transfers/test_presto_to_slack.py       |  48 ++++-
 .../providers/slack/transfers}/__init__.py         |   0
 .../providers/slack/transfers/test_sql_to_slack.py | 213 +++++++++++++++++++++
 .../snowflake/transfers/test_snowflake_to_slack.py |  96 +++++++---
 .../providers/presto/example_presto_to_slack.py    |   2 +
 .../system/providers/slack}/__init__.py            |   0
 .../example_sql_to_slack.py}                       |  19 +-
 19 files changed, 525 insertions(+), 241 deletions(-)
 copy airflow/{api_connexion => providers/slack/transfers}/__init__.py (100%)
 copy airflow/providers/{snowflake/transfers/snowflake_to_slack.py => slack/transfers/sql_to_slack.py} (54%)
 copy docs/{apache-airflow-providers-presto/operators/transfer/presto_to_slack.rst => apache-airflow-providers-slack/operators/sql_to_slack.rst} (64%)
 copy {airflow/api/auth => tests/providers/slack/transfers}/__init__.py (100%)
 create mode 100644 tests/providers/slack/transfers/test_sql_to_slack.py
 copy {airflow/api_connexion => tests/system/providers/slack}/__init__.py (100%)
 copy tests/system/providers/{presto/example_presto_to_slack.py => slack/example_sql_to_slack.py} (75%)


[airflow] 01/01: Move provider dependencies to inside provider folders

Posted by po...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

potiuk pushed a commit to branch move-setup-dependencies-to-providers
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 6349b5e578fe8cbcce3bba1782f08e0287e7169e
Author: Jarek Potiuk <ja...@polidea.com>
AuthorDate: Sun Jun 26 20:23:13 2022 +0200

    Move provider dependencies to inside provider folders
    
    The ``setup.py`` had ALWAUS contained provider dependencies,
    but this is really a remnant of Airlfow 1.10 where providers
    were not separated out to subfolders of "providers".
    
    This change moves all the provider-specific dependencies
    to provider.yaml where they are kept together with all other
    provider meta-data.
    
    Later, when we move providers out, we can move them to
    provider specific setup.py files (or let provider-specific
    setup.py files read them from provider.yaml) but this is
    not something we want to do it now.
    
    The dependencies.json is now renamed to provider_dependencies.json
    and moved to "airflow" so tha it can be kept as part of the
    sources needed for sdist package to provide extras. Pre-commit still
    generates the file as needed and it contains now both:
    
    * cross-provider-deps information which providers depend on each
      other
    * deps - information what regular dependencies are needed for each
      provider
    
    On top of preparing to splitting providers it has the advantage,
    that there will be no more case where adding a dependency change
    for provider will not run tests for that provider.
---
 .dockerignore                                      |   2 +-
 .github/workflows/ci.yml                           |   2 +-
 .pre-commit-config.yaml                            |  16 +-
 CONTRIBUTING.rst                                   |  65 +-
 Dockerfile.ci                                      |   2 +-
 MANIFEST.in                                        |   1 +
 STATIC_CODE_CHECKS.rst                             |   2 -
 airflow/dependencies.json                          | 817 +++++++++++++++++++++
 airflow/provider.yaml.schema.json                  |  36 +-
 airflow/providers/airbyte/provider.yaml            |   3 +-
 airflow/providers/alibaba/provider.yaml            |   3 +-
 airflow/providers/amazon/provider.yaml             |  15 +-
 airflow/providers/apache/beam/provider.yaml        |   7 +-
 airflow/providers/apache/cassandra/provider.yaml   |   3 +-
 airflow/providers/apache/drill/provider.yaml       |   4 +-
 airflow/providers/apache/druid/provider.yaml       |   3 +-
 airflow/providers/apache/hdfs/provider.yaml        |   4 +-
 airflow/providers/apache/hive/provider.yaml        |  10 +-
 airflow/providers/apache/kylin/provider.yaml       |   3 +-
 airflow/providers/apache/livy/provider.yaml        |   3 +-
 airflow/providers/apache/pig/provider.yaml         |   2 +-
 airflow/providers/apache/pinot/provider.yaml       |   5 +-
 airflow/providers/apache/spark/provider.yaml       |   3 +-
 airflow/providers/apache/sqoop/provider.yaml       |   2 +-
 airflow/providers/arangodb/provider.yaml           |   3 +-
 airflow/providers/asana/provider.yaml              |   3 +-
 airflow/providers/celery/provider.yaml             |   8 +-
 airflow/providers/cloudant/provider.yaml           |   3 +-
 airflow/providers/cncf/kubernetes/provider.yaml    |  12 +-
 airflow/providers/core/sql/provider.yaml           |   2 +-
 airflow/providers/databricks/provider.yaml         |   5 +-
 airflow/providers/datadog/provider.yaml            |   3 +-
 airflow/providers/dbt/cloud/provider.yaml          |   3 +-
 airflow/providers/dependencies.json                |  95 ---
 airflow/providers/dingding/provider.yaml           |   3 +-
 airflow/providers/discord/provider.yaml            |   3 +-
 airflow/providers/docker/provider.yaml             |   3 +-
 airflow/providers/elasticsearch/provider.yaml      |   5 +-
 airflow/providers/exasol/provider.yaml             |   4 +-
 airflow/providers/facebook/provider.yaml           |   3 +-
 airflow/providers/ftp/provider.yaml                |   2 +
 airflow/providers/github/provider.yaml             |   3 +-
 airflow/providers/google/provider.yaml             |  76 +-
 airflow/providers/grpc/provider.yaml               |  11 +-
 airflow/providers/hashicorp/provider.yaml          |   3 +-
 airflow/providers/http/provider.yaml               |   5 +
 airflow/providers/imap/provider.yaml               |   3 +
 airflow/providers/influxdb/provider.yaml           |   4 +-
 airflow/providers/jdbc/provider.yaml               |   3 +-
 airflow/providers/jenkins/provider.yaml            |   3 +-
 airflow/providers/jira/provider.yaml               |   3 +-
 airflow/providers/microsoft/azure/provider.yaml    |  20 +-
 airflow/providers/microsoft/mssql/provider.yaml    |   3 +-
 airflow/providers/microsoft/psrp/provider.yaml     |   2 +-
 airflow/providers/microsoft/winrm/provider.yaml    |   3 +-
 airflow/providers/mongo/provider.yaml              |   6 +-
 airflow/providers/mysql/provider.yaml              |   4 +-
 airflow/providers/neo4j/provider.yaml              |   3 +-
 airflow/providers/odbc/provider.yaml               |   3 +-
 airflow/providers/openfaas/provider.yaml           |   2 +-
 airflow/providers/opsgenie/provider.yaml           |   3 +-
 airflow/providers/oracle/provider.yaml             |   7 +-
 airflow/providers/pagerduty/provider.yaml          |   3 +-
 airflow/providers/papermill/provider.yaml          |   4 +-
 airflow/providers/plexus/provider.yaml             |   3 +-
 airflow/providers/postgres/provider.yaml           |   3 +-
 airflow/providers/presto/provider.yaml             |   4 +-
 airflow/providers/qubole/provider.yaml             |   3 +-
 airflow/providers/redis/provider.yaml              |   7 +-
 airflow/providers/salesforce/provider.yaml         |   5 +-
 airflow/providers/samba/provider.yaml              |   3 +-
 airflow/providers/segment/provider.yaml            |   3 +-
 airflow/providers/sendgrid/provider.yaml           |   3 +-
 airflow/providers/sftp/provider.yaml               |   2 +-
 airflow/providers/singularity/provider.yaml        |   3 +-
 airflow/providers/slack/provider.yaml              |   4 +-
 airflow/providers/snowflake/provider.yaml          |   4 +-
 airflow/providers/sqlite/provider.yaml             |   2 +
 airflow/providers/ssh/provider.yaml                |   4 +-
 airflow/providers/tableau/provider.yaml            |   3 +-
 airflow/providers/telegram/provider.yaml           |   3 +-
 airflow/providers/trino/provider.yaml              |   4 +-
 airflow/providers/vertica/provider.yaml            |   3 +-
 airflow/providers/yandex/provider.yaml             |   3 +-
 airflow/providers/zendesk/provider.yaml            |   3 +-
 dev/breeze/src/airflow_breeze/pre_commit_ids.py    |   1 -
 .../src/airflow_breeze/utils/selective_checks.py   |   2 +
 dev/breeze/tests/test_selective_checks.py          |  16 +
 dev/provider_packages/prepare_provider_packages.py | 100 +--
 .../howto/create-update-providers.rst              |  26 +-
 generated/README.md                                |  25 +
 generated/provider_dependencies.json               | 675 +++++++++++++++++
 images/breeze/output-commands-hash.txt             |   2 +-
 images/breeze/output-static-checks.svg             | 228 +++---
 scripts/ci/docker-compose/remove-sources.yml       |   5 +-
 scripts/ci/pre_commit/common_precommit_utils.py    |   6 +-
 .../pre_commit_build_providers_dependencies.py     | 201 +++++
 .../pre_commit_build_providers_dependencies.sh     |  36 -
 .../pre_commit_check_extras_have_providers.py      |  86 ---
 .../ci/pre_commit/pre_commit_check_order_setup.py  |   5 +-
 .../pre_commit_check_setup_extra_packages_ref.py   |   8 +-
 scripts/ci/pre_commit/pre_commit_insert_extras.py  |   6 +-
 scripts/in_container/_in_container_utils.sh        |   2 +-
 scripts/in_container/run_generate_constraints.sh   |   2 +-
 scripts/in_container/verify_providers.py           |  14 +-
 setup.cfg                                          |   4 +
 setup.py                                           | 624 ++++------------
 tests/build_provider_packages_dependencies.py      | 280 -------
 108 files changed, 2433 insertions(+), 1343 deletions(-)

diff --git a/.dockerignore b/.dockerignore
index 69a3bbfca6..73cf1cf88b 100644
--- a/.dockerignore
+++ b/.dockerignore
@@ -74,12 +74,12 @@
 !setup.cfg
 !setup.py
 !manifests
+!generated
 # Now - ignore unnecessary files inside allowed directories
 # This goes after the allowed directories
 
 # Git version is dynamically generated
 airflow/git_version
-
 # Exclude static www files generated by NPM
 airflow/www/static/coverage
 airflow/www/static/dist
diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index 87ce6ece3a..7a8337d88d 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -840,7 +840,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
         if: always()
 
   prepare-test-provider-packages-sdist:
-    timeout-minutes: 40
+    timeout-minutes: 80
     name: "Build and test provider packages sdist"
     runs-on: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     needs: [build-info, wait-for-ci-images]
diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index bcb89ea708..dfddb91bd3 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -363,11 +363,11 @@ repos:
         pass_filenames: false
       - id: update-providers-dependencies
         name: Update cross-dependencies for providers packages
-        entry: ./scripts/ci/pre_commit/pre_commit_build_providers_dependencies.sh
+        entry: ./scripts/ci/pre_commit/pre_commit_build_providers_dependencies.py
         language: python
-        files: ^airflow/providers/.*\.py$|^tests/providers/.*\.py$
+        files: ^airflow/providers/.*\.py$|^tests/providers/.*\.py$|^tests/system/providers/.*\.py$|$airflow/providers/.*/provider.yaml$
         pass_filenames: false
-        additional_dependencies: ['setuptools']
+        additional_dependencies: ['setuptools', 'rich>=12.4.4', 'pyyaml']
       - id: update-extras
         name: Update extras in documentation
         entry: ./scripts/ci/pre_commit/pre_commit_insert_extras.py
@@ -620,14 +620,6 @@ repos:
         additional_dependencies: ['pyyaml', 'jinja2', 'black==22.3.0', 'tabulate', 'rich>=12.4.4']
         require_serial: true
         pass_filenames: false
-      - id: check-airflow-providers-have-extras
-        name: Checks providers available when declared by extras in setup.py
-        language: python
-        entry: ./scripts/ci/pre_commit/pre_commit_check_extras_have_providers.py
-        files: ^setup\.py$|^airflow/providers/.*\.py$
-        pass_filenames: false
-        require_serial: true
-        additional_dependencies: ['rich>=12.4.4']
       - id: update-breeze-readme-config-hash
         name: Update Breeze README.md with config files hash
         language: python
@@ -863,5 +855,5 @@ repos:
         entry: ./scripts/ci/pre_commit/pre_commit_migration_reference.py
         pass_filenames: false
         files: ^airflow/migrations/versions/.*\.py$|^docs/apache-airflow/migrations-ref\.rst$
-        additional_dependencies: ['rich>=12.4.4', 'inputimeout']
+        additional_dependencies: ['rich>=12.4.4', 'inputimeout', 'markdown-it-py']
         ## ONLY ADD PRE-COMMITS HERE THAT REQUIRE CI IMAGE
diff --git a/CONTRIBUTING.rst b/CONTRIBUTING.rst
index c2e5d53ede..283e430aed 100644
--- a/CONTRIBUTING.rst
+++ b/CONTRIBUTING.rst
@@ -637,7 +637,23 @@ Provider packages
 Airflow 2.0 is split into core and providers. They are delivered as separate packages:
 
 * ``apache-airflow`` - core of Apache Airflow
-* ``apache-airflow-providers-*`` - More than 50 provider packages to communicate with external services
+* ``apache-airflow-providers-*`` - More than 70 provider packages to communicate with external services
+
+The information/meta-data about the providers is kept in ``provider.yaml`` file in the right sub-directory
+of ``airflow\providers``. This file contains:
+
+* package name (``apache-airflow-provider-*``)
+* user-facing name of the provider package
+* description of the package that is available in the documentation
+* list of versions of package that have been released so far
+* list of dependencies of the provider package
+* list of additional-extras that the provider package provides (together with dependencies of those extras)
+* list of integrations, operators, hooks, sensors, transfers provided by the provider (useful for documentation generation)
+* list of connection types, extra-links, secret backends, auth backends, and logging handlers (useful to both
+  register them as they are needed by Airflow and to include them in documentation automatically).
+
+If you want to add dependencies to the provider, you should add them to the corresponding ``provider.yaml``
+and Airflow pre-commits and package generation commands will use them when preparing package information.
 
 In Airflow 1.10 all those providers were installed together within one single package and when you installed
 airflow locally, from sources, they were also installed. In Airflow 2.0, providers are separated out,
@@ -656,7 +672,7 @@ in this airflow folder - the providers package is importable.
 Some of the packages have cross-dependencies with other providers packages. This typically happens for
 transfer operators where operators use hooks from the other providers in case they are transferring
 data between the providers. The list of dependencies is maintained (automatically with pre-commits)
-in the ``airflow/providers/dependencies.json``. Pre-commits are also used to generate dependencies.
+in the ``generated/provider_dependencies.json``. Pre-commits are also used to generate dependencies.
 The dependency list is automatically used during PyPI packages generation.
 
 Cross-dependencies between provider packages are converted into extras - if you need functionality from
@@ -666,49 +682,8 @@ the other provider package you can install it adding [extra] after the
 transfer operators from Amazon ECS.
 
 If you add a new dependency between different providers packages, it will be detected automatically during
-pre-commit phase and pre-commit will fail - and add entry in dependencies.json so that the package extra
-dependencies are properly added when package is installed.
-
-You can regenerate the whole list of provider dependencies by running this command (you need to have
-``pre-commits`` installed).
-
-.. code-block:: bash
-
-  pre-commit run build-providers-dependencies
-
-
-Here is the list of packages and their extras:
-
-
-  .. START PACKAGE DEPENDENCIES HERE
-
-========================== ===========================
-Package                    Extras
-========================== ===========================
-airbyte                    http
-amazon                     apache.hive,cncf.kubernetes,exasol,ftp,google,imap,mongo,salesforce,ssh
-apache.beam                google
-apache.druid               apache.hive
-apache.hive                amazon,microsoft.mssql,mysql,presto,samba,vertica
-apache.livy                http
-dbt.cloud                  http
-dingding                   http
-discord                    http
-google                     amazon,apache.beam,apache.cassandra,cncf.kubernetes,facebook,microsoft.azure,microsoft.mssql,mysql,oracle,postgres,presto,salesforce,sftp,ssh,trino
-hashicorp                  google
-microsoft.azure            google,oracle,sftp
-mysql                      amazon,presto,trino,vertica
-postgres                   amazon
-presto                     google,slack
-salesforce                 tableau
-sftp                       ssh
-slack                      http
-snowflake                  slack
-trino                      google
-========================== ===========================
-
-  .. END PACKAGE DEPENDENCIES HERE
-
+and pre-commit will generate new entry in ``generated/provider_dependencies.json`` so that
+the package extra dependencies are properly handled when package is installed.
 
 Developing community managed provider packages
 ----------------------------------------------
diff --git a/Dockerfile.ci b/Dockerfile.ci
index e9f3f4b428..06ada2775d 100644
--- a/Dockerfile.ci
+++ b/Dockerfile.ci
@@ -1321,8 +1321,8 @@ RUN REMOVE_ARTIFACTS="false" BUILD_TYPE="build" bash /scripts/docker/compile_www
 # So in case setup.py changes we can install latest dependencies required.
 COPY setup.py ${AIRFLOW_SOURCES}/setup.py
 COPY setup.cfg ${AIRFLOW_SOURCES}/setup.cfg
-
 COPY airflow/__init__.py ${AIRFLOW_SOURCES}/airflow/
+COPY generated/provider_dependencies.json ${AIRFLOW_SOURCES}/generated/
 
 COPY --from=scripts install_airflow.sh /scripts/docker/
 
diff --git a/MANIFEST.in b/MANIFEST.in
index 8f4b22b7ca..bfcaf6057c 100644
--- a/MANIFEST.in
+++ b/MANIFEST.in
@@ -38,3 +38,4 @@ include airflow/customized_form_field_behaviours.schema.json
 include airflow/serialization/schema.json
 include airflow/utils/python_virtualenv_script.jinja2
 include airflow/utils/context.pyi
+include generated
diff --git a/STATIC_CODE_CHECKS.rst b/STATIC_CODE_CHECKS.rst
index 8190695b07..45d17dde18 100644
--- a/STATIC_CODE_CHECKS.rst
+++ b/STATIC_CODE_CHECKS.rst
@@ -140,8 +140,6 @@ require Breeze Docker image to be build locally.
 +--------------------------------------------------------+------------------------------------------------------------------+---------+
 | check-airflow-config-yaml-consistent                   | Checks for consistency between config.yml and default_config.cfg |         |
 +--------------------------------------------------------+------------------------------------------------------------------+---------+
-| check-airflow-providers-have-extras                    | Checks providers available when declared by extras in setup.py   |         |
-+--------------------------------------------------------+------------------------------------------------------------------+---------+
 | check-apache-license-rat                               | Check if licenses are OK for Apache                              |         |
 +--------------------------------------------------------+------------------------------------------------------------------+---------+
 | check-base-operator-partial-arguments                  | Check BaseOperator and partial() arguments                       |         |
diff --git a/airflow/dependencies.json b/airflow/dependencies.json
new file mode 100644
index 0000000000..4a49bd3d45
--- /dev/null
+++ b/airflow/dependencies.json
@@ -0,0 +1,817 @@
+{
+  "airbyte": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "apache-airflow-providers-http"
+      ]
+    ],
+    "cross-providers-deps": [
+      "http"
+    ]
+  },
+  "alibaba": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "oss2>=2.14.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "amazon": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "boto3>=1.15.0",
+        "watchtower~=2.0.1",
+        "jsonpath_ng>=1.5.3",
+        "redshift_connector>=2.0.888",
+        "sqlalchemy_redshift>=0.8.6",
+        "pandas>=0.17.1",
+        "mypy-boto3-rds>=1.21.0'",
+        "mypy-boto3-redshift-data>=1.21.0",
+        "mypy-boto3-appflow>=1.21.0"
+      ]
+    ],
+    "cross-providers-deps": [
+      "apache.hive",
+      "cncf.kubernetes",
+      "exasol",
+      "ftp",
+      "google",
+      "imap",
+      "mongo",
+      "salesforce",
+      "ssh"
+    ]
+  },
+  "apache.beam": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "apache-beam>=2.39.0"
+      ]
+    ],
+    "cross-providers-deps": [
+      "google"
+    ]
+  },
+  "apache.cassandra": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "cassandra-driver>=3.13.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "apache.drill": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "sqlalchemy-drill>=1.1.0",
+        "sqlparse>=0.4.1"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "apache.druid": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "pydruid>=0.4.1"
+      ]
+    ],
+    "cross-providers-deps": [
+      "apache.hive"
+    ]
+  },
+  "apache.hdfs": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "snakebite-py3",
+        "hdfs[avro,dataframe,kerberos]>=2.0.4"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "apache.hive": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "hmsclient>=0.1.0",
+        "pandas>=0.17.1",
+        "pyhive[hive]>=0.6.0",
+        "sasl>=0.3.1; python_version>=\"3.9\"",
+        "thrift>=0.9.2"
+      ]
+    ],
+    "cross-providers-deps": [
+      "amazon",
+      "microsoft.mssql",
+      "mysql",
+      "presto",
+      "samba",
+      "vertica"
+    ]
+  },
+  "apache.kylin": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "kylinpy>=2.6"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "apache.livy": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "apache-airflow-providers-http"
+      ]
+    ],
+    "cross-providers-deps": [
+      "http"
+    ]
+  },
+  "apache.pig": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "apache.pinot": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "pinotdb>0.1.2"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "apache.spark": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "pyspark"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "apache.sqoop": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "arangodb": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "python-arango>=7.3.2"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "asana": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "asana>=0.10"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "celery": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "celery>=5.2.3,<6",
+        "flower>=1.0.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "cloudant": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "cloudant>=2.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "cncf.kubernetes": {
+    "deps": [
+      [
+        "apache-airflow>=2.3.0",
+        "cryptography>=2.0.0",
+        "kubernetes>=21.7.0,<24"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "databricks": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "requests>=2.27,<3",
+        "databricks-sql-connector>=2.0.0, <3.0.0",
+        "aiohttp>=3.6.3, <4"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "datadog": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "datadog>=0.14.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "dbt.cloud": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "apache-airflow-providers-http"
+      ]
+    ],
+    "cross-providers-deps": [
+      "http"
+    ]
+  },
+  "dingding": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "apache-airflow-providers-http"
+      ]
+    ],
+    "cross-providers-deps": [
+      "http"
+    ]
+  },
+  "discord": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "apache-airflow-providers-http"
+      ]
+    ],
+    "cross-providers-deps": [
+      "http"
+    ]
+  },
+  "docker": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "docker>=5.0.3"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "elasticsearch": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "elasticsearch>7",
+        "elasticsearch-dbapi",
+        "elasticsearch-dsl>=5.0.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "exasol": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "pyexasol>=0.5.1",
+        "pandas>=0.17.1"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "facebook": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "facebook-business>=6.0.2"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "ftp": {
+    "deps": [
+      []
+    ],
+    "cross-providers-deps": []
+  },
+  "github": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "pygithub"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "google": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "PyOpenSSL",
+        "google-ads>=15.1.1",
+        "google-api-core>=2.7.0,<3.0.0",
+        "google-api-python-client>=1.6.0,<2.0.0",
+        "google-auth>=1.0.0",
+        "google-auth-httplib2>=0.0.1",
+        "google-cloud-aiplatform>=1.7.1,<2.0.0",
+        "google-cloud-automl>=2.1.0",
+        "google-cloud-bigquery-datatransfer>=3.0.0",
+        "google-cloud-bigtable>=1.0.0,<2.0.0",
+        "google-cloud-build>=3.0.0",
+        "google-cloud-container>=2.2.0,<3.0.0",
+        "google-cloud-datacatalog>=3.0.0",
+        "google-cloud-dataplex>=0.1.0",
+        "google-cloud-dataproc>=3.1.0",
+        "google-cloud-dataproc-metastore>=1.2.0,<2.0.0",
+        "google-cloud-dlp>=0.11.0,<2.0.0",
+        "google-cloud-kms>=2.0.0",
+        "google-cloud-language>=1.1.1,<2.0.0",
+        "google-cloud-logging>=2.1.1",
+        "google-cloud-memcache>=0.2.0",
+        "google-cloud-monitoring>=2.0.0",
+        "google-cloud-os-login>=2.0.0",
+        "google-cloud-orchestration-airflow>=1.0.0,<2.0.0",
+        "google-cloud-pubsub>=2.0.0",
+        "google-cloud-redis>=2.0.0",
+        "google-cloud-secret-manager>=0.2.0,<2.0.0",
+        "google-cloud-spanner>=1.10.0,<2.0.0",
+        "google-cloud-speech>=0.36.3,<2.0.0",
+        "google-cloud-storage>=1.30,<2.0.0",
+        "google-cloud-tasks>=2.0.0",
+        "google-cloud-texttospeech>=0.4.0,<2.0.0",
+        "google-cloud-translate>=1.5.0,<2.0.0",
+        "google-cloud-videointelligence>=1.7.0,<2.0.0",
+        "google-cloud-vision>=0.35.2,<2.0.0",
+        "google-cloud-workflows>=0.1.0,<2.0.0",
+        "grpcio-gcp>=0.2.2",
+        "httpx",
+        "json-merge-patch>=0.2",
+        "looker-sdk>=22.2.0",
+        "pandas-gbq",
+        "pandas>=0.17.1",
+        "sqlalchemy-bigquery>=1.2.1",
+        "proto-plus>=1.19.6"
+      ]
+    ],
+    "cross-providers-deps": [
+      "amazon",
+      "apache.beam",
+      "apache.cassandra",
+      "cncf.kubernetes",
+      "facebook",
+      "microsoft.azure",
+      "microsoft.mssql",
+      "mysql",
+      "oracle",
+      "postgres",
+      "presto",
+      "salesforce",
+      "sftp",
+      "ssh",
+      "trino"
+    ]
+  },
+  "grpc": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "google-auth>=1.0.0, <3.0.0",
+        "google-auth-httplib2>=0.0.1",
+        "grpcio>=1.15.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "hashicorp": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "hvac>=0.10"
+      ]
+    ],
+    "cross-providers-deps": [
+      "google"
+    ]
+  },
+  "http": {
+    "deps": [
+      [
+        "requests>=2.26.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "imap": {
+    "deps": [
+      []
+    ],
+    "cross-providers-deps": []
+  },
+  "influxdb": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "influxdb-client>=1.19.0",
+        "requests>=2.26.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "jdbc": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "jaydebeapi>=1.1.1"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "jenkins": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "python-jenkins>=1.0.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "jira": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "JIRA>1.0.7"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "microsoft.azure": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "azure-batch>=8.0.0',",
+        "azure-cosmos>=4.0.0",
+        "azure-datalake-store>=0.0.45",
+        "azure-identity>=1.3.1",
+        "azure-keyvault-secrets>=4.1.0,<5.0",
+        "azure-kusto-data>=0.0.43,<0.1",
+        "azure-mgmt-containerinstance>=1.5.0,<2.0",
+        "azure-mgmt-datafactory>=1.0.0,<2.0",
+        "azure-mgmt-datalake-store>=0.5.0",
+        "azure-mgmt-resource>=2.2.0",
+        "azure-storage-blob>=12.7.0,<12.9.0",
+        "azure-storage-common>=2.1.0",
+        "azure-storage-file>=2.1.0",
+        "azure-servicebus>=7.6.1; platform_machine != \"aarch64\""
+      ]
+    ],
+    "cross-providers-deps": [
+      "google",
+      "oracle",
+      "sftp"
+    ]
+  },
+  "microsoft.mssql": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "pymssql>=2.1.5; platform_machine != \"aarch64"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "microsoft.psrp": {
+    "deps": [
+      [
+        "pypsrp>=0.8.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "microsoft.winrm": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "pywinrm>=0.4"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "mongo": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "dnspython>=1.13.0",
+        "pymongo>=3.6.0,<4.0.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "mysql": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "mysql-connector-python>=8.0.11; platform_machine != \"aarch64\"",
+        "mysqlclient>=1.3.6; platform_machine != \"aarch64\""
+      ]
+    ],
+    "cross-providers-deps": [
+      "amazon",
+      "presto",
+      "trino",
+      "vertica"
+    ]
+  },
+  "neo4j": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "neo4j>=4.2.1"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "odbc": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "pyodbc"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "openfaas": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "opsgenie": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "opsgenie-sdk>=2.1.5"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "oracle": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "oracledb>=1.0.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "pagerduty": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "pdpyras>=4.1.2"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "papermill": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "papermill[all]>=1.2.1",
+        "scrapbook[all]"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "plexus": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "arrow>=0.16.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "postgres": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "psycopg2-binary>=2.7.4"
+      ]
+    ],
+    "cross-providers-deps": [
+      "amazon"
+    ]
+  },
+  "presto": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "presto-python-client>=0.8.2",
+        "pandas>=0.17.1"
+      ]
+    ],
+    "cross-providers-deps": [
+      "google",
+      "slack"
+    ]
+  },
+  "qubole": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "qds-sdk>=1.10.4"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "redis": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "redis~=3.2"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "salesforce": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "simple-salesforce>=1.0.0",
+        "tableauserverclient",
+        "pandas>=0.17.1"
+      ]
+    ],
+    "cross-providers-deps": [
+      "tableau"
+    ]
+  },
+  "samba": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "smbprotocol>=1.5.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "segment": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "analytics-python>=1.2.9"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "sendgrid": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "sendgrid>=6.0.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "sftp": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "apache-airflow-providers-ssh>=2.1.0"
+      ]
+    ],
+    "cross-providers-deps": [
+      "ssh"
+    ]
+  },
+  "singularity": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "spython>=0.0.56"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "slack": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "apache-airflow-providers-http",
+        "slack_sdk>=3.0.0"
+      ]
+    ],
+    "cross-providers-deps": [
+      "http"
+    ]
+  },
+  "snowflake": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "snowflake-connector-python>=2.4.1",
+        "snowflake-sqlalchemy>=1.1.0"
+      ]
+    ],
+    "cross-providers-deps": [
+      "slack"
+    ]
+  },
+  "sqlite": {
+    "deps": [
+      []
+    ],
+    "cross-providers-deps": []
+  },
+  "ssh": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "paramiko>=2.6.0",
+        "sshtunnel>=0.3.2"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "tableau": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "tableauserverclient"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "telegram": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "python-telegram-bot>=13.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "trino": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "pandas>=0.17.1",
+        "trino>=0.301.0"
+      ]
+    ],
+    "cross-providers-deps": [
+      "google"
+    ]
+  },
+  "vertica": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "vertica-python>=0.5.1"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "yandex": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "yandexcloud>=0.146.0"
+      ]
+    ],
+    "cross-providers-deps": []
+  },
+  "zendesk": {
+    "deps": [
+      [
+        "apache-airflow>=2.2.0",
+        "zenpy>=2.0.24"
+      ]
+    ],
+    "cross-providers-deps": []
+  }
+}
diff --git a/airflow/provider.yaml.schema.json b/airflow/provider.yaml.schema.json
index c7e02d1d15..ff0537db32 100644
--- a/airflow/provider.yaml.schema.json
+++ b/airflow/provider.yaml.schema.json
@@ -21,8 +21,8 @@
         "type": "string"
       }
     },
-    "additional-dependencies": {
-      "description": "Additional dependencies that should be added to the provider",
+    "dependencies": {
+      "description": "Dependencies that should be added to the provider",
       "type": "array",
       "items": {
         "type": "string"
@@ -208,9 +208,12 @@
                   "description": "Hook class name that implements the connection type",
                   "type": "string"
               }
-          }
-      },
-      "required": ["connection-type", "hook-class-name"]
+          },
+          "required": [
+              "connection-type",
+              "hook-class-name"
+          ]
+      }
     },
     "extra-links": {
       "type": "array",
@@ -220,8 +223,26 @@
       }
     },
     "additional-extras": {
-      "type": "object",
-      "description": "Additional extras that the provider should have"
+      "type": "array",
+      "items": {
+        "type": "object",
+        "properties": {
+          "name": {
+            "description": "Name of the extra",
+            "type": "string"
+          },
+          "dependencies": {
+            "description": "Dependencies that should be added for the extra",
+            "type": "array",
+            "items": {
+              "type": "string"
+            }
+          }
+        },
+        "required": [ "name", "dependencies"]
+      },
+
+      "description": "Additional extras that the provider should have. Replaces auto-generated cross-provider extras, if matching the same prefix, so that you can specify boundaries for existing dependencies."
     },
     "task-decorators": {
         "type": "array",
@@ -262,6 +283,7 @@
     "name",
     "package-name",
     "description",
+    "dependencies",
     "versions"
   ]
 }
diff --git a/airflow/providers/airbyte/provider.yaml b/airflow/providers/airbyte/provider.yaml
index 46e786ecd0..d65bdf88af 100644
--- a/airflow/providers/airbyte/provider.yaml
+++ b/airflow/providers/airbyte/provider.yaml
@@ -31,8 +31,9 @@ versions:
   - 2.0.0
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-http
 
 integrations:
   - integration-name: Airbyte
diff --git a/airflow/providers/alibaba/provider.yaml b/airflow/providers/alibaba/provider.yaml
index 18bac8d8fb..d858e5ed57 100644
--- a/airflow/providers/alibaba/provider.yaml
+++ b/airflow/providers/alibaba/provider.yaml
@@ -28,8 +28,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - oss2>=2.14.0
 
 integrations:
   - integration-name: Alibaba Cloud OSS
diff --git a/airflow/providers/amazon/provider.yaml b/airflow/providers/amazon/provider.yaml
index 321986a3e7..ee7f246298 100644
--- a/airflow/providers/amazon/provider.yaml
+++ b/airflow/providers/amazon/provider.yaml
@@ -41,8 +41,21 @@ versions:
   - 1.1.0
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - boto3>=1.15.0
+  # watchtower 3 has been released end Jan and introduced breaking change across the board that might
+  # change logging behaviour:
+  # https://github.com/kislyuk/watchtower/blob/develop/Changes.rst#changes-for-v300-2022-01-26
+  # TODO: update to watchtower >3
+  - watchtower~=2.0.1
+  - jsonpath_ng>=1.5.3
+  - redshift_connector>=2.0.888
+  - sqlalchemy_redshift>=0.8.6
+  - pandas>=0.17.1
+  - mypy-boto3-rds>=1.21.0
+  - mypy-boto3-redshift-data>=1.21.0
+  - mypy-boto3-appflow>=1.21.0
 
 integrations:
   - integration-name: Amazon Athena
diff --git a/airflow/providers/apache/beam/provider.yaml b/airflow/providers/apache/beam/provider.yaml
index 4d06f21208..f106e53476 100644
--- a/airflow/providers/apache/beam/provider.yaml
+++ b/airflow/providers/apache/beam/provider.yaml
@@ -34,8 +34,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - apache-beam>=2.39.0
 
 integrations:
   - integration-name: Apache Beam
@@ -55,4 +56,6 @@ hooks:
       - airflow.providers.apache.beam.hooks.beam
 
 additional-extras:
-  google: apache-beam[gcp]
+  - name: google
+    dependencies:
+      - apache-beam[gcp]
diff --git a/airflow/providers/apache/cassandra/provider.yaml b/airflow/providers/apache/cassandra/provider.yaml
index f902472428..c376066d47 100644
--- a/airflow/providers/apache/cassandra/provider.yaml
+++ b/airflow/providers/apache/cassandra/provider.yaml
@@ -32,8 +32,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - cassandra-driver>=3.13.0
 
 integrations:
   - integration-name: Apache Cassandra
diff --git a/airflow/providers/apache/drill/provider.yaml b/airflow/providers/apache/drill/provider.yaml
index 4f50d00c97..dcdf128e50 100644
--- a/airflow/providers/apache/drill/provider.yaml
+++ b/airflow/providers/apache/drill/provider.yaml
@@ -29,8 +29,10 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - sqlalchemy-drill>=1.1.0
+  - sqlparse>=0.4.1
 
 integrations:
   - integration-name: Apache Drill
diff --git a/airflow/providers/apache/druid/provider.yaml b/airflow/providers/apache/druid/provider.yaml
index 415d216953..a1af570a15 100644
--- a/airflow/providers/apache/druid/provider.yaml
+++ b/airflow/providers/apache/druid/provider.yaml
@@ -36,8 +36,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - pydruid>=0.4.1
 
 integrations:
   - integration-name: Apache Druid
diff --git a/airflow/providers/apache/hdfs/provider.yaml b/airflow/providers/apache/hdfs/provider.yaml
index 5016b6786f..e7e2f65851 100644
--- a/airflow/providers/apache/hdfs/provider.yaml
+++ b/airflow/providers/apache/hdfs/provider.yaml
@@ -34,8 +34,10 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - snakebite-py3
+  - hdfs[avro,dataframe,kerberos]>=2.0.4
 
 integrations:
   - integration-name: Hadoop Distributed File System (HDFS)
diff --git a/airflow/providers/apache/hive/provider.yaml b/airflow/providers/apache/hive/provider.yaml
index 2da854c441..cd9321c4cd 100644
--- a/airflow/providers/apache/hive/provider.yaml
+++ b/airflow/providers/apache/hive/provider.yaml
@@ -38,8 +38,16 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - hmsclient>=0.1.0
+  - pandas>=0.17.1
+  - pyhive[hive]>=0.6.0
+  # in case of Python 3.9 sasl library needs to be installed with version higher or equal than
+  # 0.3.1 because only that version supports Python 3.9. For other Python version pyhive[hive] pulls
+  # the sasl library anyway (and there sasl library version is not relevant)
+  - sasl>=0.3.1; python_version>="3.9"
+  - thrift>=0.9.2
 
 integrations:
   - integration-name: Apache Hive
diff --git a/airflow/providers/apache/kylin/provider.yaml b/airflow/providers/apache/kylin/provider.yaml
index 8c5e5e3a7b..fbdbf9ef00 100644
--- a/airflow/providers/apache/kylin/provider.yaml
+++ b/airflow/providers/apache/kylin/provider.yaml
@@ -31,8 +31,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - kylinpy>=2.6
 
 integrations:
   - integration-name: Apache Kylin
diff --git a/airflow/providers/apache/livy/provider.yaml b/airflow/providers/apache/livy/provider.yaml
index 662c311f09..54f99980f5 100644
--- a/airflow/providers/apache/livy/provider.yaml
+++ b/airflow/providers/apache/livy/provider.yaml
@@ -33,8 +33,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-http
 
 integrations:
   - integration-name: Apache Livy
diff --git a/airflow/providers/apache/pig/provider.yaml b/airflow/providers/apache/pig/provider.yaml
index 626209769c..8b1ad67daa 100644
--- a/airflow/providers/apache/pig/provider.yaml
+++ b/airflow/providers/apache/pig/provider.yaml
@@ -31,7 +31,7 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
 
 integrations:
diff --git a/airflow/providers/apache/pinot/provider.yaml b/airflow/providers/apache/pinot/provider.yaml
index 035af4978e..8c2708c2fb 100644
--- a/airflow/providers/apache/pinot/provider.yaml
+++ b/airflow/providers/apache/pinot/provider.yaml
@@ -31,8 +31,11 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  # pinotdb v0.1.1 may still work with older versions of Apache Pinot, but we've confirmed that it
+  # causes a problem with newer versions.
+  - pinotdb>0.1.2
 
 integrations:
   - integration-name: Apache Pinot
diff --git a/airflow/providers/apache/spark/provider.yaml b/airflow/providers/apache/spark/provider.yaml
index 18f9cacef1..629ed74d39 100644
--- a/airflow/providers/apache/spark/provider.yaml
+++ b/airflow/providers/apache/spark/provider.yaml
@@ -36,8 +36,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - pyspark
 
 integrations:
   - integration-name: Apache Spark
diff --git a/airflow/providers/apache/sqoop/provider.yaml b/airflow/providers/apache/sqoop/provider.yaml
index 1e34f75ec9..db7e8e7e9a 100644
--- a/airflow/providers/apache/sqoop/provider.yaml
+++ b/airflow/providers/apache/sqoop/provider.yaml
@@ -33,7 +33,7 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
 
 integrations:
diff --git a/airflow/providers/arangodb/provider.yaml b/airflow/providers/arangodb/provider.yaml
index 7c28494ee1..09cc3f27f5 100644
--- a/airflow/providers/arangodb/provider.yaml
+++ b/airflow/providers/arangodb/provider.yaml
@@ -21,8 +21,9 @@ name: ArangoDB
 description: |
     `ArangoDB <https://www.arangodb.com/>`__
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - python-arango>=7.3.2
 
 versions:
   - 2.0.0
diff --git a/airflow/providers/asana/provider.yaml b/airflow/providers/asana/provider.yaml
index 9391fe93e3..87ebb3218f 100644
--- a/airflow/providers/asana/provider.yaml
+++ b/airflow/providers/asana/provider.yaml
@@ -29,8 +29,9 @@ versions:
   - 1.1.0
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - asana>=0.10
 
 integrations:
   - integration-name: Asana
diff --git a/airflow/providers/celery/provider.yaml b/airflow/providers/celery/provider.yaml
index 2e2f5bb90a..53fe5a96a9 100644
--- a/airflow/providers/celery/provider.yaml
+++ b/airflow/providers/celery/provider.yaml
@@ -32,8 +32,14 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  # The Celery is known to introduce problems when upgraded to a MAJOR version. Airflow Core
+  # Uses Celery for CeleryExecutor, and we also know that Kubernetes Python client follows SemVer
+  # (https://docs.celeryq.dev/en/stable/contributing.html?highlight=semver#versions).
+  # Make sure that the limit here is synchronized with [celery] extra in the airflow core
+  - celery>=5.2.3,<6
+  - flower>=1.0.0
 
 integrations:
   - integration-name: Celery
diff --git a/airflow/providers/cloudant/provider.yaml b/airflow/providers/cloudant/provider.yaml
index 3b0065dca1..91225e4de6 100644
--- a/airflow/providers/cloudant/provider.yaml
+++ b/airflow/providers/cloudant/provider.yaml
@@ -31,8 +31,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - cloudant>=2.0
 
 integrations:
   - integration-name: IBM Cloudant
diff --git a/airflow/providers/cncf/kubernetes/provider.yaml b/airflow/providers/cncf/kubernetes/provider.yaml
index 0205de3004..ece009bac0 100644
--- a/airflow/providers/cncf/kubernetes/provider.yaml
+++ b/airflow/providers/cncf/kubernetes/provider.yaml
@@ -44,8 +44,18 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.3.0
+  - cryptography>=2.0.0
+  # The Kubernetes API is known to introduce problems when upgraded to a MAJOR version. Airflow Core
+  # Uses Kubernetes for Kubernetes executor, and we also know that Kubernetes Python client follows SemVer
+  # (https://github.com/kubernetes-client/python#compatibility). This is a crucial component of Airflow
+  # So we should limit it to the next MAJOR version and only deliberately bump the version when we
+  # tested it, and we know it can be bumped. Bumping this version should also be connected with
+  # limiting minimum airflow version supported in cncf.kubernetes provider, due to the
+  # potential breaking changes in Airflow Core as well (kubernetes is added as extra, so Airflow
+  # core is not hard-limited via install-requirements, only by extra).
+  - kubernetes>=21.7.0,<24
 
 integrations:
   - integration-name: Kubernetes
diff --git a/airflow/providers/core/sql/provider.yaml b/airflow/providers/core/sql/provider.yaml
index 64f6d9746c..f951984267 100644
--- a/airflow/providers/core/sql/provider.yaml
+++ b/airflow/providers/core/sql/provider.yaml
@@ -24,7 +24,7 @@ description: |
 versions:
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
 
 integrations:
diff --git a/airflow/providers/databricks/provider.yaml b/airflow/providers/databricks/provider.yaml
index 6136e6fe6d..d20878bef5 100644
--- a/airflow/providers/databricks/provider.yaml
+++ b/airflow/providers/databricks/provider.yaml
@@ -36,8 +36,11 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - requests>=2.27,<3
+  - databricks-sql-connector>=2.0.0, <3.0.0
+  - aiohttp>=3.6.3, <4
 
 integrations:
   - integration-name: Databricks
diff --git a/airflow/providers/datadog/provider.yaml b/airflow/providers/datadog/provider.yaml
index 3496509220..5daf04bd69 100644
--- a/airflow/providers/datadog/provider.yaml
+++ b/airflow/providers/datadog/provider.yaml
@@ -31,8 +31,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - datadog>=0.14.0
 
 integrations:
   - integration-name: Datadog
diff --git a/airflow/providers/dbt/cloud/provider.yaml b/airflow/providers/dbt/cloud/provider.yaml
index 28bf2060c4..c61a47f026 100644
--- a/airflow/providers/dbt/cloud/provider.yaml
+++ b/airflow/providers/dbt/cloud/provider.yaml
@@ -26,8 +26,9 @@ versions:
   - 1.0.2
   - 1.0.1
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-http
 
 integrations:
   - integration-name: dbt Cloud
diff --git a/airflow/providers/dependencies.json b/airflow/providers/dependencies.json
deleted file mode 100644
index dd58aac77d..0000000000
--- a/airflow/providers/dependencies.json
+++ /dev/null
@@ -1,95 +0,0 @@
-{
-  "airbyte": [
-    "http"
-  ],
-  "amazon": [
-    "apache.hive",
-    "cncf.kubernetes",
-    "exasol",
-    "ftp",
-    "google",
-    "imap",
-    "mongo",
-    "salesforce",
-    "ssh"
-  ],
-  "apache.beam": [
-    "google"
-  ],
-  "apache.druid": [
-    "apache.hive"
-  ],
-  "apache.hive": [
-    "amazon",
-    "microsoft.mssql",
-    "mysql",
-    "presto",
-    "samba",
-    "vertica"
-  ],
-  "apache.livy": [
-    "http"
-  ],
-  "dbt.cloud": [
-    "http"
-  ],
-  "dingding": [
-    "http"
-  ],
-  "discord": [
-    "http"
-  ],
-  "google": [
-    "amazon",
-    "apache.beam",
-    "apache.cassandra",
-    "cncf.kubernetes",
-    "facebook",
-    "microsoft.azure",
-    "microsoft.mssql",
-    "mysql",
-    "oracle",
-    "postgres",
-    "presto",
-    "salesforce",
-    "sftp",
-    "ssh",
-    "trino"
-  ],
-  "hashicorp": [
-    "google"
-  ],
-  "microsoft.azure": [
-    "google",
-    "oracle",
-    "sftp"
-  ],
-  "mysql": [
-    "amazon",
-    "presto",
-    "trino",
-    "vertica"
-  ],
-  "postgres": [
-    "amazon"
-  ],
-  "presto": [
-    "google",
-    "slack"
-  ],
-  "salesforce": [
-    "tableau"
-  ],
-  "sftp": [
-    "ssh"
-  ],
-  "slack": [
-    "http"
-  ],
-  "snowflake": [
-    "slack"
-  ],
-  "trino": [
-    "google"
-  ]
-}
diff --git a/airflow/providers/dingding/provider.yaml b/airflow/providers/dingding/provider.yaml
index e2d283974b..8032435b36 100644
--- a/airflow/providers/dingding/provider.yaml
+++ b/airflow/providers/dingding/provider.yaml
@@ -32,8 +32,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-http
 
 integrations:
   - integration-name: Dingding
diff --git a/airflow/providers/discord/provider.yaml b/airflow/providers/discord/provider.yaml
index 779fde70a6..f5f0d650ff 100644
--- a/airflow/providers/discord/provider.yaml
+++ b/airflow/providers/discord/provider.yaml
@@ -32,8 +32,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-http
 
 integrations:
   - integration-name: Discord
diff --git a/airflow/providers/docker/provider.yaml b/airflow/providers/docker/provider.yaml
index de5fe97119..dad4524836 100644
--- a/airflow/providers/docker/provider.yaml
+++ b/airflow/providers/docker/provider.yaml
@@ -41,8 +41,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - docker>=5.0.3
 
 integrations:
   - integration-name: Docker
diff --git a/airflow/providers/elasticsearch/provider.yaml b/airflow/providers/elasticsearch/provider.yaml
index b6753e2fc7..e9f4a2ddca 100644
--- a/airflow/providers/elasticsearch/provider.yaml
+++ b/airflow/providers/elasticsearch/provider.yaml
@@ -38,8 +38,11 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - elasticsearch>7
+  - elasticsearch-dbapi
+  - elasticsearch-dsl>=5.0.0
 
 integrations:
   - integration-name: Elasticsearch
diff --git a/airflow/providers/exasol/provider.yaml b/airflow/providers/exasol/provider.yaml
index a861c4dc73..b0594fb653 100644
--- a/airflow/providers/exasol/provider.yaml
+++ b/airflow/providers/exasol/provider.yaml
@@ -33,8 +33,10 @@ versions:
   - 1.1.0
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - pyexasol>=0.5.1
+  - pandas>=0.17.1
 
 integrations:
   - integration-name: Exasol
diff --git a/airflow/providers/facebook/provider.yaml b/airflow/providers/facebook/provider.yaml
index 572f9763b0..94f78b63f9 100644
--- a/airflow/providers/facebook/provider.yaml
+++ b/airflow/providers/facebook/provider.yaml
@@ -34,8 +34,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - facebook-business>=6.0.2
 
 integrations:
   - integration-name: Facebook Ads
diff --git a/airflow/providers/ftp/provider.yaml b/airflow/providers/ftp/provider.yaml
index f8e6df16e3..57f5a20ef3 100644
--- a/airflow/providers/ftp/provider.yaml
+++ b/airflow/providers/ftp/provider.yaml
@@ -32,6 +32,8 @@ versions:
   - 1.0.1
   - 1.0.0
 
+dependencies: []
+
 integrations:
   - integration-name: File Transfer Protocol (FTP)
     external-doc-url: https://tools.ietf.org/html/rfc114
diff --git a/airflow/providers/github/provider.yaml b/airflow/providers/github/provider.yaml
index 08265ba0c8..a1e40b1cd8 100644
--- a/airflow/providers/github/provider.yaml
+++ b/airflow/providers/github/provider.yaml
@@ -22,8 +22,9 @@ name: Github
 description: |
     `GitHub <https://www.github.com/>`__
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - pygithub
 
 versions:
   - 2.0.0
diff --git a/airflow/providers/google/provider.yaml b/airflow/providers/google/provider.yaml
index 8b361da130..225333d543 100644
--- a/airflow/providers/google/provider.yaml
+++ b/airflow/providers/google/provider.yaml
@@ -50,8 +50,62 @@ versions:
   - 2.0.0
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  # Google has very clear rules on what dependencies should be used. All the limits below
+  # follow strict guidelines of Google Libraries as quoted here:
+  # While this issue is open, dependents of google-api-core, google-cloud-core. and google-auth
+  # should preserve >1, <3 pins on these packages.
+  # https://github.com/googleapis/google-cloud-python/issues/10566
+  # Some of Google Packages are limited to <2.0.0 because 2.0.0 releases of the libraries
+  # Introduced breaking changes across the board. Those libraries should be upgraded soon
+  # TODO: Upgrade all Google libraries that are limited to <2.0.0
+  - PyOpenSSL
+  - google-ads>=15.1.1
+  - google-api-core>=2.7.0,<3.0.0
+  - google-api-python-client>=1.6.0,<2.0.0
+  - google-auth>=1.0.0
+  - google-auth-httplib2>=0.0.1
+  - google-cloud-aiplatform>=1.7.1,<2.0.0
+  - google-cloud-automl>=2.1.0
+  - google-cloud-bigquery-datatransfer>=3.0.0
+  - google-cloud-bigtable>=1.0.0,<2.0.0
+  - google-cloud-build>=3.0.0
+  - google-cloud-container>=2.2.0,<3.0.0
+  - google-cloud-datacatalog>=3.0.0
+  - google-cloud-dataplex>=0.1.0
+  - google-cloud-dataproc>=3.1.0
+  - google-cloud-dataproc-metastore>=1.2.0,<2.0.0
+  - google-cloud-dlp>=0.11.0,<2.0.0
+  - google-cloud-kms>=2.0.0
+  - google-cloud-language>=1.1.1,<2.0.0
+  - google-cloud-logging>=2.1.1
+  - google-cloud-memcache>=0.2.0
+  - google-cloud-monitoring>=2.0.0
+  - google-cloud-os-login>=2.0.0
+  - google-cloud-orchestration-airflow>=1.0.0,<2.0.0
+  - google-cloud-pubsub>=2.0.0
+  - google-cloud-redis>=2.0.0
+  - google-cloud-secret-manager>=0.2.0,<2.0.0
+  - google-cloud-spanner>=1.10.0,<2.0.0
+  - google-cloud-speech>=0.36.3,<2.0.0
+  - google-cloud-storage>=1.30,<2.0.0
+  - google-cloud-tasks>=2.0.0
+  - google-cloud-texttospeech>=0.4.0,<2.0.0
+  - google-cloud-translate>=1.5.0,<2.0.0
+  - google-cloud-videointelligence>=1.7.0,<2.0.0
+  - google-cloud-vision>=0.35.2,<2.0.0
+  - google-cloud-workflows>=0.1.0,<2.0.0
+  - grpcio-gcp>=0.2.2
+  - httpx
+  - json-merge-patch>=0.2
+  - looker-sdk>=22.2.0
+  - pandas-gbq
+  - pandas>=0.17.1
+  - sqlalchemy-bigquery>=1.2.1
+  # A transient dependency of google-cloud-bigquery-datatransfer, but we
+  # further constrain it since older versions are buggy.
+  - proto-plus>=1.19.6
 
 integrations:
   - integration-name: Google Analytics360
@@ -921,11 +975,21 @@ extra-links:
   - airflow.providers.google.common.links.storage.FileDetailsLink
 
 additional-extras:
-  apache.beam: apache-beam[gcp]
-  leveldb: plyvel
-  oracle: apache-airflow-providers-oracle>=3.1.0
-  facebook: apache-airflow-providers-facebook>=2.2.0
-  amazon: apache-airflow-providers-amazon>=2.6.0
+  - name: apache.beam
+    dependencies:
+      - apache-beam[gcp]
+  - name: leveldb
+    dependencies:
+      - plyvel
+  - name: oracle
+    dependencies:
+      - apache-airflow-providers-oracle>=3.1.0
+  - name: facebook
+    dependencies:
+      - apache-airflow-providers-facebook>=2.2.0
+  - name: amazon
+    dependencies:
+      - apache-airflow-providers-amazon>=2.6.0
 
 secrets-backends:
   - airflow.providers.google.cloud.secrets.secret_manager.CloudSecretManagerBackend
diff --git a/airflow/providers/grpc/provider.yaml b/airflow/providers/grpc/provider.yaml
index 33dee46c64..133aa0c334 100644
--- a/airflow/providers/grpc/provider.yaml
+++ b/airflow/providers/grpc/provider.yaml
@@ -31,9 +31,16 @@ versions:
   - 1.1.0
   - 1.0.1
   - 1.0.0
-
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  # Google has very clear rules on what dependencies should be used. All the limits below
+  # follow strict guidelines of Google Libraries as quoted here:
+  # While this issue is open, dependents of google-api-core, google-cloud-core. and google-auth
+  # should preserve >1, <3 pins on these packages.
+  # https://github.com/googleapis/google-cloud-python/issues/10566
+  - google-auth>=1.0.0, <3.0.0
+  - google-auth-httplib2>=0.0.1
+  - grpcio>=1.15.0
 
 integrations:
   - integration-name: gRPC
diff --git a/airflow/providers/hashicorp/provider.yaml b/airflow/providers/hashicorp/provider.yaml
index c92e49f7a8..e811fe98ab 100644
--- a/airflow/providers/hashicorp/provider.yaml
+++ b/airflow/providers/hashicorp/provider.yaml
@@ -34,8 +34,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - hvac>=0.10
 
 integrations:
   - integration-name: Hashicorp Vault
diff --git a/airflow/providers/http/provider.yaml b/airflow/providers/http/provider.yaml
index 01b18b4cbe..4437744507 100644
--- a/airflow/providers/http/provider.yaml
+++ b/airflow/providers/http/provider.yaml
@@ -34,6 +34,11 @@ versions:
   - 1.1.0
   - 1.0.0
 
+dependencies:
+  # The 2.26.0 release of requests got rid of the chardet LGPL mandatory dependency, allowing us to
+  # release it as a requirement for airflow
+  - requests>=2.26.0
+
 integrations:
   - integration-name: Hypertext Transfer Protocol (HTTP)
     external-doc-url: https://www.w3.org/Protocols/
diff --git a/airflow/providers/imap/provider.yaml b/airflow/providers/imap/provider.yaml
index 354aeab473..dd3a791c68 100644
--- a/airflow/providers/imap/provider.yaml
+++ b/airflow/providers/imap/provider.yaml
@@ -18,6 +18,7 @@
 ---
 package-name: apache-airflow-providers-imap
 name: Internet Message Access Protocol (IMAP)
+
 description: |
     `Internet Message Access Protocol (IMAP) <https://tools.ietf.org/html/rfc3501>`__
 
@@ -33,6 +34,8 @@ versions:
   - 1.0.1
   - 1.0.0
 
+dependencies: []
+
 integrations:
   - integration-name: Internet Message Access Protocol (IMAP)
     external-doc-url: https://tools.ietf.org/html/rfc3501
diff --git a/airflow/providers/influxdb/provider.yaml b/airflow/providers/influxdb/provider.yaml
index bd348fab3e..628beee808 100644
--- a/airflow/providers/influxdb/provider.yaml
+++ b/airflow/providers/influxdb/provider.yaml
@@ -23,8 +23,10 @@ name: Influxdb
 description: |
     `InfluxDB <https://www.influxdata.com/>`__
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - influxdb-client>=1.19.0
+  - requests>=2.26.0
 
 versions:
   - 2.0.0
diff --git a/airflow/providers/jdbc/provider.yaml b/airflow/providers/jdbc/provider.yaml
index 4c4631d1b5..f12f371ab8 100644
--- a/airflow/providers/jdbc/provider.yaml
+++ b/airflow/providers/jdbc/provider.yaml
@@ -32,8 +32,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - jaydebeapi>=1.1.1
 
 integrations:
   - integration-name: Java Database Connectivity (JDBC)
diff --git a/airflow/providers/jenkins/provider.yaml b/airflow/providers/jenkins/provider.yaml
index 6bc3cb06fa..8f8068f576 100644
--- a/airflow/providers/jenkins/provider.yaml
+++ b/airflow/providers/jenkins/provider.yaml
@@ -36,8 +36,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - python-jenkins>=1.0.0
 
 integrations:
   - integration-name: Jenkins
diff --git a/airflow/providers/jira/provider.yaml b/airflow/providers/jira/provider.yaml
index f47abdbfee..d56d971ffd 100644
--- a/airflow/providers/jira/provider.yaml
+++ b/airflow/providers/jira/provider.yaml
@@ -33,8 +33,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - JIRA>1.0.7
 
 integrations:
   - integration-name: Atlassian Jira
diff --git a/airflow/providers/microsoft/azure/provider.yaml b/airflow/providers/microsoft/azure/provider.yaml
index 5e87ae5488..101bbd9a2f 100644
--- a/airflow/providers/microsoft/azure/provider.yaml
+++ b/airflow/providers/microsoft/azure/provider.yaml
@@ -41,8 +41,26 @@ versions:
   - 1.1.0
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - azure-batch>=8.0.0
+  - azure-cosmos>=4.0.0
+  - azure-datalake-store>=0.0.45
+  - azure-identity>=1.3.1
+  - azure-keyvault-secrets>=4.1.0,<5.0
+  - azure-kusto-data>=0.0.43,<0.1
+  # Azure integration uses old libraries and the limits below reflect that
+  # TODO: upgrade to newer versions of all the below libraries
+  - azure-mgmt-containerinstance>=1.5.0,<2.0
+  - azure-mgmt-datafactory>=1.0.0,<2.0
+  - azure-mgmt-datalake-store>=0.5.0
+  - azure-mgmt-resource>=2.2.0
+  # limited due to https://github.com/Azure/azure-sdk-for-python/pull/18801  implementation released in 12.9
+  - azure-storage-blob>=12.7.0,<12.9.0
+  - azure-storage-common>=2.1.0
+  - azure-storage-file>=2.1.0
+  # Limited due to https://github.com/Azure/azure-uamqp-python/issues/191
+  - azure-servicebus>=7.6.1; platform_machine != "aarch64"
 
 integrations:
   - integration-name: Microsoft Azure Batch
diff --git a/airflow/providers/microsoft/mssql/provider.yaml b/airflow/providers/microsoft/mssql/provider.yaml
index d8729510dd..b2feb654fd 100644
--- a/airflow/providers/microsoft/mssql/provider.yaml
+++ b/airflow/providers/microsoft/mssql/provider.yaml
@@ -33,8 +33,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - pymssql>=2.1.5; platform_machine != "aarch64"
 
 integrations:
   - integration-name: Microsoft SQL Server (MSSQL)
diff --git a/airflow/providers/microsoft/psrp/provider.yaml b/airflow/providers/microsoft/psrp/provider.yaml
index fc868a0594..84d9171419 100644
--- a/airflow/providers/microsoft/psrp/provider.yaml
+++ b/airflow/providers/microsoft/psrp/provider.yaml
@@ -33,7 +33,7 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - pypsrp>=0.8.0
 
 integrations:
diff --git a/airflow/providers/microsoft/winrm/provider.yaml b/airflow/providers/microsoft/winrm/provider.yaml
index 8e732e06c5..96c37d0e41 100644
--- a/airflow/providers/microsoft/winrm/provider.yaml
+++ b/airflow/providers/microsoft/winrm/provider.yaml
@@ -34,8 +34,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - pywinrm>=0.4
 
 integrations:
   - integration-name: Windows Remote Management (WinRM)
diff --git a/airflow/providers/mongo/provider.yaml b/airflow/providers/mongo/provider.yaml
index bd3ee31b71..a7fdd51e64 100644
--- a/airflow/providers/mongo/provider.yaml
+++ b/airflow/providers/mongo/provider.yaml
@@ -33,8 +33,12 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - dnspython>=1.13.0
+  # pymongo 4.0.0 removes connection option `ssl_cert_reqs` which is used in providers-mongo/2.2.0
+  # TODO: Upgrade to pymongo 4.0.0+
+  - pymongo>=3.6.0,<4.0.0
 
 integrations:
   - integration-name: MongoDB
diff --git a/airflow/providers/mysql/provider.yaml b/airflow/providers/mysql/provider.yaml
index 7c6e80a6a6..21fa45f649 100644
--- a/airflow/providers/mysql/provider.yaml
+++ b/airflow/providers/mysql/provider.yaml
@@ -35,8 +35,10 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - mysql-connector-python>=8.0.11; platform_machine != "aarch64"
+  - mysqlclient>=1.3.6; platform_machine != "aarch64"
 
 integrations:
   - integration-name: MySQL
diff --git a/airflow/providers/neo4j/provider.yaml b/airflow/providers/neo4j/provider.yaml
index f20116750b..a5e87ad4bc 100644
--- a/airflow/providers/neo4j/provider.yaml
+++ b/airflow/providers/neo4j/provider.yaml
@@ -33,8 +33,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - neo4j>=4.2.1
 
 integrations:
   - integration-name: Neo4j
diff --git a/airflow/providers/odbc/provider.yaml b/airflow/providers/odbc/provider.yaml
index 6ecbd2ae64..12dc46f5eb 100644
--- a/airflow/providers/odbc/provider.yaml
+++ b/airflow/providers/odbc/provider.yaml
@@ -31,8 +31,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - pyodbc
 
 integrations:
   - integration-name: ODBC
diff --git a/airflow/providers/openfaas/provider.yaml b/airflow/providers/openfaas/provider.yaml
index f52a3cdfca..3962a571c9 100644
--- a/airflow/providers/openfaas/provider.yaml
+++ b/airflow/providers/openfaas/provider.yaml
@@ -31,7 +31,7 @@ versions:
   - 1.1.0
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
 
 integrations:
diff --git a/airflow/providers/opsgenie/provider.yaml b/airflow/providers/opsgenie/provider.yaml
index 6beecd150c..255f8ecf46 100644
--- a/airflow/providers/opsgenie/provider.yaml
+++ b/airflow/providers/opsgenie/provider.yaml
@@ -34,8 +34,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - opsgenie-sdk>=2.1.5
 
 integrations:
   - integration-name: Opsgenie
diff --git a/airflow/providers/oracle/provider.yaml b/airflow/providers/oracle/provider.yaml
index 9ff438ca0e..513cc34fc0 100644
--- a/airflow/providers/oracle/provider.yaml
+++ b/airflow/providers/oracle/provider.yaml
@@ -35,8 +35,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - oracledb>=1.0.0
 
 integrations:
   - integration-name: Oracle
@@ -45,7 +46,9 @@ integrations:
     tags: [software]
 
 additional-extras:
-  numpy: numpy
+  - name: numpy
+    dependencies:
+      - numpy
 
 operators:
   - integration-name: Oracle
diff --git a/airflow/providers/pagerduty/provider.yaml b/airflow/providers/pagerduty/provider.yaml
index aac14c6dbc..537bca625c 100644
--- a/airflow/providers/pagerduty/provider.yaml
+++ b/airflow/providers/pagerduty/provider.yaml
@@ -32,8 +32,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - pdpyras>=4.1.2
 
 integrations:
   - integration-name: Pagerduty
diff --git a/airflow/providers/papermill/provider.yaml b/airflow/providers/papermill/provider.yaml
index d9ef31d6c7..fcc9a0543d 100644
--- a/airflow/providers/papermill/provider.yaml
+++ b/airflow/providers/papermill/provider.yaml
@@ -34,8 +34,10 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - papermill[all]>=1.2.1
+  - scrapbook[all]
 
 integrations:
   - integration-name: Papermill
diff --git a/airflow/providers/plexus/provider.yaml b/airflow/providers/plexus/provider.yaml
index 0f4f39b1a0..1b9f5054ea 100644
--- a/airflow/providers/plexus/provider.yaml
+++ b/airflow/providers/plexus/provider.yaml
@@ -31,8 +31,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - arrow>=0.16.0
 
 integrations:
   - integration-name: Plexus
diff --git a/airflow/providers/postgres/provider.yaml b/airflow/providers/postgres/provider.yaml
index 99a4d75c60..9c4c70ad3e 100644
--- a/airflow/providers/postgres/provider.yaml
+++ b/airflow/providers/postgres/provider.yaml
@@ -36,8 +36,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - psycopg2-binary>=2.7.4
 
 integrations:
   - integration-name: PostgreSQL
diff --git a/airflow/providers/presto/provider.yaml b/airflow/providers/presto/provider.yaml
index c2eb2f32ec..26d7db6f8a 100644
--- a/airflow/providers/presto/provider.yaml
+++ b/airflow/providers/presto/provider.yaml
@@ -34,8 +34,10 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - presto-python-client>=0.8.2
+  - pandas>=0.17.1
 
 integrations:
   - integration-name: Presto
diff --git a/airflow/providers/qubole/provider.yaml b/airflow/providers/qubole/provider.yaml
index 395fdc0fdb..1826d2299c 100644
--- a/airflow/providers/qubole/provider.yaml
+++ b/airflow/providers/qubole/provider.yaml
@@ -33,8 +33,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - qds-sdk>=1.10.4
 
 integrations:
   - integration-name: Qubole
diff --git a/airflow/providers/redis/provider.yaml b/airflow/providers/redis/provider.yaml
index 4debd216c4..15c353d8d8 100644
--- a/airflow/providers/redis/provider.yaml
+++ b/airflow/providers/redis/provider.yaml
@@ -31,8 +31,13 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  # Redis 4 introduced a number of changes that likely need testing including mixins in redis commands
+  # as well as unquoting URLS with `urllib.parse.unquote`:
+  # https://github.com/redis/redis-py/blob/master/CHANGES
+  # TODO: upgrade to support redis package >=4
+  - redis~=3.2
 
 integrations:
   - integration-name: Redis
diff --git a/airflow/providers/salesforce/provider.yaml b/airflow/providers/salesforce/provider.yaml
index 363bdfc09c..5e1d2dee04 100644
--- a/airflow/providers/salesforce/provider.yaml
+++ b/airflow/providers/salesforce/provider.yaml
@@ -36,8 +36,11 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - simple-salesforce>=1.0.0
+  - tableauserverclient
+  - pandas>=0.17.1
 
 integrations:
   - integration-name: Salesforce
diff --git a/airflow/providers/samba/provider.yaml b/airflow/providers/samba/provider.yaml
index 4d796173ac..dd9c8674ca 100644
--- a/airflow/providers/samba/provider.yaml
+++ b/airflow/providers/samba/provider.yaml
@@ -32,8 +32,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - smbprotocol>=1.5.0
 
 integrations:
   - integration-name: Samba
diff --git a/airflow/providers/segment/provider.yaml b/airflow/providers/segment/provider.yaml
index 69dafca7e8..0cede0faa0 100644
--- a/airflow/providers/segment/provider.yaml
+++ b/airflow/providers/segment/provider.yaml
@@ -31,8 +31,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - analytics-python>=1.2.9
 
 integrations:
   - integration-name: Segment
diff --git a/airflow/providers/sendgrid/provider.yaml b/airflow/providers/sendgrid/provider.yaml
index ebc0ac5ad9..8c8fe945b0 100644
--- a/airflow/providers/sendgrid/provider.yaml
+++ b/airflow/providers/sendgrid/provider.yaml
@@ -21,8 +21,9 @@ name: Sendgrid
 description: |
     `Sendgrid <https://sendgrid.com/>`__
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - sendgrid>=6.0.0
 
 versions:
   - 3.0.0
diff --git a/airflow/providers/sftp/provider.yaml b/airflow/providers/sftp/provider.yaml
index 2143377359..3f6b9fdc29 100644
--- a/airflow/providers/sftp/provider.yaml
+++ b/airflow/providers/sftp/provider.yaml
@@ -39,7 +39,7 @@ versions:
   - 1.1.0
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
   - apache-airflow-providers-ssh>=2.1.0
 
diff --git a/airflow/providers/singularity/provider.yaml b/airflow/providers/singularity/provider.yaml
index 2124997621..800bc02653 100644
--- a/airflow/providers/singularity/provider.yaml
+++ b/airflow/providers/singularity/provider.yaml
@@ -32,8 +32,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - spython>=0.0.56
 
 integrations:
   - integration-name: Singularity
diff --git a/airflow/providers/slack/provider.yaml b/airflow/providers/slack/provider.yaml
index cfd1ef89b7..2a8331d1ac 100644
--- a/airflow/providers/slack/provider.yaml
+++ b/airflow/providers/slack/provider.yaml
@@ -34,8 +34,10 @@ versions:
   - 2.0.0
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-http
+  - slack_sdk>=3.0.0
 
 integrations:
   - integration-name: Slack
diff --git a/airflow/providers/snowflake/provider.yaml b/airflow/providers/snowflake/provider.yaml
index 99dcd1f1a4..c1c035bde4 100644
--- a/airflow/providers/snowflake/provider.yaml
+++ b/airflow/providers/snowflake/provider.yaml
@@ -41,8 +41,10 @@ versions:
   - 1.1.0
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - snowflake-connector-python>=2.4.1
+  - snowflake-sqlalchemy>=1.1.0
 
 integrations:
   - integration-name: Snowflake
diff --git a/airflow/providers/sqlite/provider.yaml b/airflow/providers/sqlite/provider.yaml
index ad5f90e706..d663a06e3c 100644
--- a/airflow/providers/sqlite/provider.yaml
+++ b/airflow/providers/sqlite/provider.yaml
@@ -33,6 +33,8 @@ versions:
   - 1.0.1
   - 1.0.0
 
+dependencies: []
+
 integrations:
   - integration-name: SQLite
     external-doc-url: https://www.sqlite.org/index.html
diff --git a/airflow/providers/ssh/provider.yaml b/airflow/providers/ssh/provider.yaml
index ce7732c2fa..dafbd2e636 100644
--- a/airflow/providers/ssh/provider.yaml
+++ b/airflow/providers/ssh/provider.yaml
@@ -38,8 +38,10 @@ versions:
   - 1.1.0
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - paramiko>=2.6.0
+  - sshtunnel>=0.3.2
 
 integrations:
   - integration-name: Secure Shell (SSH)
diff --git a/airflow/providers/tableau/provider.yaml b/airflow/providers/tableau/provider.yaml
index 623d98b0db..f001e55c26 100644
--- a/airflow/providers/tableau/provider.yaml
+++ b/airflow/providers/tableau/provider.yaml
@@ -35,8 +35,9 @@ versions:
   - 2.0.0
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - tableauserverclient
 
 integrations:
   - integration-name: Tableau
diff --git a/airflow/providers/telegram/provider.yaml b/airflow/providers/telegram/provider.yaml
index 01054256be..e8097501ff 100644
--- a/airflow/providers/telegram/provider.yaml
+++ b/airflow/providers/telegram/provider.yaml
@@ -32,8 +32,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - python-telegram-bot>=13.0
 
 integrations:
   - integration-name: Telegram
diff --git a/airflow/providers/trino/provider.yaml b/airflow/providers/trino/provider.yaml
index da64337a43..f701e6c899 100644
--- a/airflow/providers/trino/provider.yaml
+++ b/airflow/providers/trino/provider.yaml
@@ -33,8 +33,10 @@ versions:
   - 2.0.0
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - pandas>=0.17.1
+  - trino>=0.301.0
 
 integrations:
   - integration-name: Trino
diff --git a/airflow/providers/vertica/provider.yaml b/airflow/providers/vertica/provider.yaml
index d7bd9b4805..0ebe1989ac 100644
--- a/airflow/providers/vertica/provider.yaml
+++ b/airflow/providers/vertica/provider.yaml
@@ -32,8 +32,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - vertica-python>=0.5.1
 
 integrations:
   - integration-name: Vertica
diff --git a/airflow/providers/yandex/provider.yaml b/airflow/providers/yandex/provider.yaml
index 88163dc35a..c066a2f8ed 100644
--- a/airflow/providers/yandex/provider.yaml
+++ b/airflow/providers/yandex/provider.yaml
@@ -32,8 +32,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - yandexcloud>=0.146.0
 
 integrations:
   - integration-name: Yandex.Cloud
diff --git a/airflow/providers/zendesk/provider.yaml b/airflow/providers/zendesk/provider.yaml
index 78c0d93263..6dc1cf264d 100644
--- a/airflow/providers/zendesk/provider.yaml
+++ b/airflow/providers/zendesk/provider.yaml
@@ -32,8 +32,9 @@ versions:
   - 1.0.1
   - 1.0.0
 
-additional-dependencies:
+dependencies:
   - apache-airflow>=2.2.0
+  - zenpy>=2.0.24
 
 integrations:
   - integration-name: Zendesk
diff --git a/dev/breeze/src/airflow_breeze/pre_commit_ids.py b/dev/breeze/src/airflow_breeze/pre_commit_ids.py
index db839f0712..0d71437653 100644
--- a/dev/breeze/src/airflow_breeze/pre_commit_ids.py
+++ b/dev/breeze/src/airflow_breeze/pre_commit_ids.py
@@ -27,7 +27,6 @@ PRE_COMMIT_LIST = [
     'blacken-docs',
     'check-airflow-2-2-compatibility',
     'check-airflow-config-yaml-consistent',
-    'check-airflow-providers-have-extras',
     'check-apache-license-rat',
     'check-base-operator-partial-arguments',
     'check-base-operator-usage',
diff --git a/dev/breeze/src/airflow_breeze/utils/selective_checks.py b/dev/breeze/src/airflow_breeze/utils/selective_checks.py
index 22e74f4553..b2d018ddd3 100644
--- a/dev/breeze/src/airflow_breeze/utils/selective_checks.py
+++ b/dev/breeze/src/airflow_breeze/utils/selective_checks.py
@@ -96,6 +96,7 @@ CI_FILE_GROUP_MATCHES = HashableDict(
             r"^scripts",
             r"^setup.py",
             r"^setup.cfg",
+            r"^generated/provider_dependencies.json$",
         ],
         FileGroupForCi.PYTHON_PRODUCTION_FILES: [
             r"^airflow/.*\.py",
@@ -119,6 +120,7 @@ CI_FILE_GROUP_MATCHES = HashableDict(
             r"^pyproject.toml",
             r"^setup.cfg",
             r"^setup.py",
+            r"^generated/provider_dependencies.json$",
         ],
         FileGroupForCi.DOC_FILES: [
             r"^docs",
diff --git a/dev/breeze/tests/test_selective_checks.py b/dev/breeze/tests/test_selective_checks.py
index 2d7e8fe83d..8b4a8e8d4d 100644
--- a/dev/breeze/tests/test_selective_checks.py
+++ b/dev/breeze/tests/test_selective_checks.py
@@ -174,6 +174,22 @@ def assert_outputs_are_printed(expected_outputs: Dict[str, str], output: str):
                 id="Everything should run and upgrading to newer requirements as setup.py changed",
             )
         ),
+        (
+            pytest.param(
+                ("generated/provider_dependencies.json",),
+                {
+                    "all-python-versions": "['3.7', '3.8', '3.9', '3.10']",
+                    "all-python-versions-list-as-string": "3.7 3.8 3.9 3.10",
+                    "image-build": "true",
+                    "needs-helm-tests": "true",
+                    "run-tests": "true",
+                    "docs-build": "true",
+                    "upgrade-to-newer-dependencies": "true",
+                    "test-types": "API Always CLI Core Integration Other Providers WWW",
+                },
+                id="Everything should run and upgrading to newer requirements as dependencies change",
+            )
+        ),
     ],
 )
 def test_expected_output_pull_request_main(
diff --git a/dev/provider_packages/prepare_provider_packages.py b/dev/provider_packages/prepare_provider_packages.py
index 44498a9aed..29d565e33c 100755
--- a/dev/provider_packages/prepare_provider_packages.py
+++ b/dev/provider_packages/prepare_provider_packages.py
@@ -92,19 +92,27 @@ AIRFLOW_PATH = AIRFLOW_SOURCES_ROOT_PATH / "airflow"
 DIST_PATH = AIRFLOW_SOURCES_ROOT_PATH / "dist"
 PROVIDERS_PATH = AIRFLOW_PATH / "providers"
 DOCUMENTATION_PATH = AIRFLOW_SOURCES_ROOT_PATH / "docs"
+
+DEPENDENCIES_JSON_FILE_PATH = AIRFLOW_SOURCES_ROOT_PATH / "generated" / "provider_dependencies.json"
+
 TARGET_PROVIDER_PACKAGES_PATH = AIRFLOW_SOURCES_ROOT_PATH / "provider_packages"
 GENERATED_AIRFLOW_PATH = TARGET_PROVIDER_PACKAGES_PATH / "airflow"
 GENERATED_PROVIDERS_PATH = GENERATED_AIRFLOW_PATH / "providers"
 
 PROVIDER_RUNTIME_DATA_SCHEMA_PATH = AIRFLOW_SOURCES_ROOT_PATH / "airflow" / "provider_info.schema.json"
 
+CROSS_PROVIDERS_DEPS = "cross-providers-deps"
+DEPS = "deps"
+
 sys.path.insert(0, str(AIRFLOW_SOURCES_ROOT_PATH))
 
+
+ALL_DEPENDENCIES = json.loads(DEPENDENCIES_JSON_FILE_PATH.read_text())
+
 # those imports need to come after the above sys.path.insert to make sure that Airflow
 # sources are importable without having to add the airflow sources to the PYTHONPATH before
 # running the script
-from setup import PROVIDERS_REQUIREMENTS  # type: ignore[attr-defined] # isort:skip # noqa
-from setup import PREINSTALLED_PROVIDERS  # type: ignore[attr-defined] # isort:skip # noqa
+from setup import PREINSTALLED_PROVIDERS, ALL_PROVIDERS  # type: ignore[attr-defined] # isort:skip # noqa
 
 # Note - we do not test protocols as they are not really part of the official API of
 # Apache Airflow
@@ -135,14 +143,6 @@ class EntityType(Enum):
     Secrets = "Secrets"
 
 
-def get_provider_packages() -> List[str]:
-    """
-    Returns all provider packages.
-
-    """
-    return list(PROVIDERS_REQUIREMENTS.keys())
-
-
 @click.group(context_settings={'help_option_names': ['-h', '--help'], 'max_content_width': 500})
 def cli():
     ...
@@ -258,9 +258,6 @@ def get_target_providers_package_folder(provider_package_id: str) -> str:
     return os.path.join(get_target_providers_folder(), *provider_package_id.split("."))
 
 
-DEPENDENCIES_JSON_FILE = os.path.join(PROVIDERS_PATH, "dependencies.json")
-
-
 def get_pip_package_name(provider_package_id: str) -> str:
     """
     Returns PIP package name for the package id.
@@ -317,28 +314,7 @@ def get_install_requirements(provider_package_id: str, version_suffix: str) -> s
 
     :return: install requirements of the package
     """
-    dependencies = PROVIDERS_REQUIREMENTS[provider_package_id]
-    provider_yaml = get_provider_yaml(provider_package_id)
-    install_requires = []
-    if "additional-dependencies" in provider_yaml:
-        additional_dependencies = provider_yaml['additional-dependencies']
-        if version_suffix:
-            # In case we are preparing "rc" or dev0 packages, we should also
-            # make sure that cross-dependency with Airflow or Airflow Providers will
-            # contain the version suffix, otherwise we will have conflicting dependencies.
-            # For example if (in sftp) we have ssh>=2.0.1 and release ssh==2.0.1
-            # we want to turn this into ssh>=2.0.1.dev0 if we build dev0 version of the packages
-            # or >=2.0.1rc1 if we build rc1 version of the packages.
-            for dependency in additional_dependencies:
-                if dependency.startswith("apache-airflow") and ">=" in dependency:
-                    dependency = (
-                        dependency + ("." if not version_suffix.startswith(".") else "") + version_suffix
-                    )
-                install_requires.append(dependency)
-        else:
-            install_requires.extend(additional_dependencies)
-
-    install_requires.extend(dependencies)
+    install_requires = ALL_DEPENDENCIES[provider_package_id][DEPS]
     prefix = "\n    "
     return prefix + prefix.join(install_requires)
 
@@ -362,24 +338,28 @@ def get_package_extras(provider_package_id: str) -> Dict[str, List[str]]:
     """
     if provider_package_id == 'providers':
         return {}
-    with open(DEPENDENCIES_JSON_FILE) as dependencies_file:
-        cross_provider_dependencies: Dict[str, List[str]] = json.load(dependencies_file)
-    extras_dict = (
-        {
-            module: [get_pip_package_name(module)]
-            for module in cross_provider_dependencies[provider_package_id]
-        }
-        if cross_provider_dependencies.get(provider_package_id)
-        else {}
-    )
+    extras_dict: Dict[str, List[str]] = {
+        module: [get_pip_package_name(module)]
+        for module in ALL_DEPENDENCIES[provider_package_id][CROSS_PROVIDERS_DEPS]
+    }
     provider_yaml_dict = get_provider_yaml(provider_package_id)
     additional_extras = provider_yaml_dict.get('additional-extras')
     if additional_extras:
-        for key in additional_extras:
-            if key in extras_dict:
-                extras_dict[key].append(additional_extras[key])
+        for entry in additional_extras:
+            name = entry['name']
+            dependencies = entry['dependencies']
+            if name in extras_dict:
+                # remove non-versioned dependencies if versioned ones are coming
+                existing_dependencies = set(extras_dict[name])
+                for new_dependency in dependencies:
+                    for dependency in existing_dependencies:
+                        # remove extra if exists as non-versioned one
+                        if new_dependency.startswith(dependency):
+                            extras_dict[name].remove(dependency)
+                            break
+                    extras_dict[name].append(new_dependency)
             else:
-                extras_dict[key] = additional_extras[key]
+                extras_dict[name] = dependencies
     return extras_dict
 
 
@@ -413,7 +393,7 @@ def render_template(
     return content
 
 
-PR_PATTERN = re.compile(r".*\(#([0-9]+)\)")
+PR_PATTERN = re.compile(r".*\(#(\d+)\)")
 
 
 class Change(NamedTuple):
@@ -660,9 +640,7 @@ def get_cross_provider_dependent_packages(provider_package_id: str) -> List[str]
     :param provider_package_id: package id
     :return: list of cross-provider dependencies
     """
-    with open(os.path.join(PROVIDERS_PATH, "dependencies.json")) as dependencies_file:
-        dependent_packages = json.load(dependencies_file).get(provider_package_id) or []
-    return dependent_packages
+    return ALL_DEPENDENCIES[provider_package_id][CROSS_PROVIDERS_DEPS]
 
 
 def make_sure_remote_apache_exists_and_fetch(git_update: bool, verbose: bool):
@@ -1023,11 +1001,7 @@ def get_provider_details(provider_package_id: str) -> ProviderPackageDetails:
 
 def get_provider_requirements(provider_package_id: str) -> List[str]:
     provider_yaml = get_provider_yaml(provider_package_id)
-    requirements = (
-        provider_yaml['additional-dependencies'].copy() if 'additional-dependencies' in provider_yaml else []
-    )
-    requirements.extend(PROVIDERS_REQUIREMENTS[provider_package_id])
-    return requirements
+    return provider_yaml['dependencies']
 
 
 def get_provider_jinja_context(
@@ -1077,7 +1051,7 @@ def get_provider_jinja_context(
             provider_package_path=provider_details.source_provider_package_path
         ),
         "CROSS_PROVIDERS_DEPENDENCIES": cross_providers_dependencies,
-        "PIP_REQUIREMENTS": PROVIDERS_REQUIREMENTS[provider_details.provider_package_id],
+        "PIP_REQUIREMENTS": get_provider_requirements(provider_details.provider_package_id),
         "PROVIDER_TYPE": "Provider",
         "PROVIDERS_FOLDER": "providers",
         "PROVIDER_DESCRIPTION": provider_details.provider_description,
@@ -1218,7 +1192,7 @@ def update_release_notes(
     :param version_suffix: version suffix corresponding to the version in the code
     :param force: regenerate already released documentation
     :param verbose: whether to print verbose messages
-    :param answer: force answer to questions if set.
+    :param answer: force answer to question if set.
     :returns False if the package should be skipped, True if everything generated properly
     """
     verify_provider_package(provider_package_id)
@@ -1449,7 +1423,7 @@ def get_all_providers() -> List[str]:
     Returns all providers for regular packages.
     :return: list of providers that are considered for provider packages
     """
-    return list(PROVIDERS_REQUIREMENTS.keys())
+    return list(ALL_PROVIDERS)
 
 
 def verify_provider_package(provider_package_id: str) -> None:
@@ -1458,10 +1432,10 @@ def verify_provider_package(provider_package_id: str) -> None:
     :param provider_package_id: package id to verify
     :return: None
     """
-    if provider_package_id not in get_provider_packages():
+    if provider_package_id not in get_all_providers():
         console.print(f"[red]Wrong package name: {provider_package_id}[/]")
         console.print("Use one of:")
-        console.print(get_provider_packages())
+        console.print(get_all_providers())
         raise Exception(f"The package {provider_package_id} is not a provider package.")
 
 
diff --git a/docs/apache-airflow-providers/howto/create-update-providers.rst b/docs/apache-airflow-providers/howto/create-update-providers.rst
index 8b974ca717..584cf3660b 100644
--- a/docs/apache-airflow-providers/howto/create-update-providers.rst
+++ b/docs/apache-airflow-providers/howto/create-update-providers.rst
@@ -130,7 +130,6 @@ Some steps for documentation occurs automatically by ``pre-commit`` see `Install
       │       └── operators/
       │           └── <NEW_PROVIDER>.rst
       └── providers/
-          ├── dependencies.json
           └── <NEW_PROVIDER>/
               ├── provider.yaml
               └── CHANGELOG.rst
@@ -138,8 +137,7 @@ Some steps for documentation occurs automatically by ``pre-commit`` see `Install
 
 Files automatically updated by pre-commit:
 
-- ``airflow/providers/dependencies.json``
-- ``INSTALL``
+- ``INSTALL`` in provider
 
 Files automatically created when the provider is released:
 
@@ -168,26 +166,8 @@ lowercase in the second block.
     nobr
     nodash
 
-Add your provider dependencies into **PROVIDER_REQUIREMENTS** variable in ``setup.py``. If your provider doesn't have
-any dependency add a empty list.
-
-  .. code-block:: python
-
-      PROVIDERS_REQUIREMENTS: Dict[str, List[str]] = {
-          # ...
-          "microsoft.winrm": winrm,
-          "mongo": mongo,
-          "mysql": mysql,
-          "neo4j": neo4j,
-          "<NEW_PROVIDER>": [],
-          "odbc": odbc,
-          # ...
-      }
-
-In the ``CONTRIBUTING.rst`` adds:
-
-- your provider name in the list in the **Extras** section
-- your provider dependencies in the **Provider Packages** section table, only if your provider has external dependencies.
+Add your provider dependencies into ``provider.yaml`` under ``dependencies`` key..
+If your provider doesn't have any dependency add a empty list.
 
 In the ``docs/apache-airflow-providers-<NEW_PROVIDER>/connections.rst``:
 
diff --git a/generated/README.md b/generated/README.md
new file mode 100644
index 0000000000..f87a767da4
--- /dev/null
+++ b/generated/README.md
@@ -0,0 +1,25 @@
+<!--
+ 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.
+ -->
+
+NOTE! The files in this folder are generated by pre-commit based on airflow sources. They are not
+supposed to be manually modified.
+
+* `provider_dependencies.json` - is generated based on `provider.yaml` files in `airflow/providers` and
+  based on the imports in the provider code. If you want to add new dependency to a provider, you
+  need to modify the corresponding `provider.yaml` file
diff --git a/generated/provider_dependencies.json b/generated/provider_dependencies.json
new file mode 100644
index 0000000000..6226703df3
--- /dev/null
+++ b/generated/provider_dependencies.json
@@ -0,0 +1,675 @@
+{
+  "airbyte": {
+    "deps": [
+      "apache-airflow-providers-http",
+      "apache-airflow>=2.2.0"
+    ],
+    "cross-providers-deps": [
+      "http"
+    ]
+  },
+  "alibaba": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "oss2>=2.14.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "amazon": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "boto3>=1.15.0",
+      "jsonpath_ng>=1.5.3",
+      "mypy-boto3-appflow>=1.21.0",
+      "mypy-boto3-rds>=1.21.0",
+      "mypy-boto3-redshift-data>=1.21.0",
+      "pandas>=0.17.1",
+      "redshift_connector>=2.0.888",
+      "sqlalchemy_redshift>=0.8.6",
+      "watchtower~=2.0.1"
+    ],
+    "cross-providers-deps": [
+      "apache.hive",
+      "cncf.kubernetes",
+      "exasol",
+      "ftp",
+      "google",
+      "imap",
+      "mongo",
+      "salesforce",
+      "ssh"
+    ]
+  },
+  "apache.beam": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "apache-beam>=2.39.0"
+    ],
+    "cross-providers-deps": [
+      "google"
+    ]
+  },
+  "apache.cassandra": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "cassandra-driver>=3.13.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "apache.drill": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "sqlalchemy-drill>=1.1.0",
+      "sqlparse>=0.4.1"
+    ],
+    "cross-providers-deps": []
+  },
+  "apache.druid": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "pydruid>=0.4.1"
+    ],
+    "cross-providers-deps": [
+      "apache.hive"
+    ]
+  },
+  "apache.hdfs": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "hdfs[avro,dataframe,kerberos]>=2.0.4",
+      "snakebite-py3"
+    ],
+    "cross-providers-deps": []
+  },
+  "apache.hive": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "hmsclient>=0.1.0",
+      "pandas>=0.17.1",
+      "pyhive[hive]>=0.6.0",
+      "sasl>=0.3.1; python_version>=\"3.9\"",
+      "thrift>=0.9.2"
+    ],
+    "cross-providers-deps": [
+      "amazon",
+      "microsoft.mssql",
+      "mysql",
+      "presto",
+      "samba",
+      "vertica"
+    ]
+  },
+  "apache.kylin": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "kylinpy>=2.6"
+    ],
+    "cross-providers-deps": []
+  },
+  "apache.livy": {
+    "deps": [
+      "apache-airflow-providers-http",
+      "apache-airflow>=2.2.0"
+    ],
+    "cross-providers-deps": [
+      "http"
+    ]
+  },
+  "apache.pig": {
+    "deps": [
+      "apache-airflow>=2.2.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "apache.pinot": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "pinotdb>0.1.2"
+    ],
+    "cross-providers-deps": []
+  },
+  "apache.spark": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "pyspark"
+    ],
+    "cross-providers-deps": []
+  },
+  "apache.sqoop": {
+    "deps": [
+      "apache-airflow>=2.2.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "arangodb": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "python-arango>=7.3.2"
+    ],
+    "cross-providers-deps": []
+  },
+  "asana": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "asana>=0.10"
+    ],
+    "cross-providers-deps": []
+  },
+  "celery": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "celery>=5.2.3,<6",
+      "flower>=1.0.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "cloudant": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "cloudant>=2.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "cncf.kubernetes": {
+    "deps": [
+      "apache-airflow>=2.3.0",
+      "cryptography>=2.0.0",
+      "kubernetes>=21.7.0,<24"
+    ],
+    "cross-providers-deps": []
+  },
+  "core.sql": {
+    "deps": [
+      "apache-airflow>=2.2.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "databricks": {
+    "deps": [
+      "aiohttp>=3.6.3, <4",
+      "apache-airflow>=2.2.0",
+      "databricks-sql-connector>=2.0.0, <3.0.0",
+      "requests>=2.27,<3"
+    ],
+    "cross-providers-deps": []
+  },
+  "datadog": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "datadog>=0.14.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "dbt.cloud": {
+    "deps": [
+      "apache-airflow-providers-http",
+      "apache-airflow>=2.2.0"
+    ],
+    "cross-providers-deps": [
+      "http"
+    ]
+  },
+  "dingding": {
+    "deps": [
+      "apache-airflow-providers-http",
+      "apache-airflow>=2.2.0"
+    ],
+    "cross-providers-deps": [
+      "http"
+    ]
+  },
+  "discord": {
+    "deps": [
+      "apache-airflow-providers-http",
+      "apache-airflow>=2.2.0"
+    ],
+    "cross-providers-deps": [
+      "http"
+    ]
+  },
+  "docker": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "docker>=5.0.3"
+    ],
+    "cross-providers-deps": []
+  },
+  "elasticsearch": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "elasticsearch-dbapi",
+      "elasticsearch-dsl>=5.0.0",
+      "elasticsearch>7"
+    ],
+    "cross-providers-deps": []
+  },
+  "exasol": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "pandas>=0.17.1",
+      "pyexasol>=0.5.1"
+    ],
+    "cross-providers-deps": []
+  },
+  "facebook": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "facebook-business>=6.0.2"
+    ],
+    "cross-providers-deps": []
+  },
+  "ftp": {
+    "deps": [],
+    "cross-providers-deps": []
+  },
+  "github": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "pygithub"
+    ],
+    "cross-providers-deps": []
+  },
+  "google": {
+    "deps": [
+      "PyOpenSSL",
+      "apache-airflow>=2.2.0",
+      "google-ads>=15.1.1",
+      "google-api-core>=2.7.0,<3.0.0",
+      "google-api-python-client>=1.6.0,<2.0.0",
+      "google-auth-httplib2>=0.0.1",
+      "google-auth>=1.0.0",
+      "google-cloud-aiplatform>=1.7.1,<2.0.0",
+      "google-cloud-automl>=2.1.0",
+      "google-cloud-bigquery-datatransfer>=3.0.0",
+      "google-cloud-bigtable>=1.0.0,<2.0.0",
+      "google-cloud-build>=3.0.0",
+      "google-cloud-container>=2.2.0,<3.0.0",
+      "google-cloud-datacatalog>=3.0.0",
+      "google-cloud-dataplex>=0.1.0",
+      "google-cloud-dataproc-metastore>=1.2.0,<2.0.0",
+      "google-cloud-dataproc>=3.1.0",
+      "google-cloud-dlp>=0.11.0,<2.0.0",
+      "google-cloud-kms>=2.0.0",
+      "google-cloud-language>=1.1.1,<2.0.0",
+      "google-cloud-logging>=2.1.1",
+      "google-cloud-memcache>=0.2.0",
+      "google-cloud-monitoring>=2.0.0",
+      "google-cloud-orchestration-airflow>=1.0.0,<2.0.0",
+      "google-cloud-os-login>=2.0.0",
+      "google-cloud-pubsub>=2.0.0",
+      "google-cloud-redis>=2.0.0",
+      "google-cloud-secret-manager>=0.2.0,<2.0.0",
+      "google-cloud-spanner>=1.10.0,<2.0.0",
+      "google-cloud-speech>=0.36.3,<2.0.0",
+      "google-cloud-storage>=1.30,<2.0.0",
+      "google-cloud-tasks>=2.0.0",
+      "google-cloud-texttospeech>=0.4.0,<2.0.0",
+      "google-cloud-translate>=1.5.0,<2.0.0",
+      "google-cloud-videointelligence>=1.7.0,<2.0.0",
+      "google-cloud-vision>=0.35.2,<2.0.0",
+      "google-cloud-workflows>=0.1.0,<2.0.0",
+      "grpcio-gcp>=0.2.2",
+      "httpx",
+      "json-merge-patch>=0.2",
+      "looker-sdk>=22.2.0",
+      "pandas-gbq",
+      "pandas>=0.17.1",
+      "proto-plus>=1.19.6",
+      "sqlalchemy-bigquery>=1.2.1"
+    ],
+    "cross-providers-deps": [
+      "amazon",
+      "apache.beam",
+      "apache.cassandra",
+      "cncf.kubernetes",
+      "facebook",
+      "microsoft.azure",
+      "microsoft.mssql",
+      "mysql",
+      "oracle",
+      "postgres",
+      "presto",
+      "salesforce",
+      "sftp",
+      "ssh",
+      "trino"
+    ]
+  },
+  "grpc": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "google-auth-httplib2>=0.0.1",
+      "google-auth>=1.0.0, <3.0.0",
+      "grpcio>=1.15.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "hashicorp": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "hvac>=0.10"
+    ],
+    "cross-providers-deps": [
+      "google"
+    ]
+  },
+  "http": {
+    "deps": [
+      "requests>=2.26.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "imap": {
+    "deps": [],
+    "cross-providers-deps": []
+  },
+  "influxdb": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "influxdb-client>=1.19.0",
+      "requests>=2.26.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "jdbc": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "jaydebeapi>=1.1.1"
+    ],
+    "cross-providers-deps": []
+  },
+  "jenkins": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "python-jenkins>=1.0.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "jira": {
+    "deps": [
+      "JIRA>1.0.7",
+      "apache-airflow>=2.2.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "microsoft.azure": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "azure-batch>=8.0.0",
+      "azure-cosmos>=4.0.0",
+      "azure-datalake-store>=0.0.45",
+      "azure-identity>=1.3.1",
+      "azure-keyvault-secrets>=4.1.0,<5.0",
+      "azure-kusto-data>=0.0.43,<0.1",
+      "azure-mgmt-containerinstance>=1.5.0,<2.0",
+      "azure-mgmt-datafactory>=1.0.0,<2.0",
+      "azure-mgmt-datalake-store>=0.5.0",
+      "azure-mgmt-resource>=2.2.0",
+      "azure-servicebus>=7.6.1; platform_machine != \"aarch64\"",
+      "azure-storage-blob>=12.7.0,<12.9.0",
+      "azure-storage-common>=2.1.0",
+      "azure-storage-file>=2.1.0"
+    ],
+    "cross-providers-deps": [
+      "google",
+      "oracle",
+      "sftp"
+    ]
+  },
+  "microsoft.mssql": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "pymssql>=2.1.5; platform_machine != \"aarch64\""
+    ],
+    "cross-providers-deps": []
+  },
+  "microsoft.psrp": {
+    "deps": [
+      "pypsrp>=0.8.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "microsoft.winrm": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "pywinrm>=0.4"
+    ],
+    "cross-providers-deps": []
+  },
+  "mongo": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "dnspython>=1.13.0",
+      "pymongo>=3.6.0,<4.0.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "mysql": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "mysql-connector-python>=8.0.11; platform_machine != \"aarch64\"",
+      "mysqlclient>=1.3.6; platform_machine != \"aarch64\""
+    ],
+    "cross-providers-deps": [
+      "amazon",
+      "presto",
+      "trino",
+      "vertica"
+    ]
+  },
+  "neo4j": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "neo4j>=4.2.1"
+    ],
+    "cross-providers-deps": []
+  },
+  "odbc": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "pyodbc"
+    ],
+    "cross-providers-deps": []
+  },
+  "openfaas": {
+    "deps": [
+      "apache-airflow>=2.2.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "opsgenie": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "opsgenie-sdk>=2.1.5"
+    ],
+    "cross-providers-deps": []
+  },
+  "oracle": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "oracledb>=1.0.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "pagerduty": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "pdpyras>=4.1.2"
+    ],
+    "cross-providers-deps": []
+  },
+  "papermill": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "papermill[all]>=1.2.1",
+      "scrapbook[all]"
+    ],
+    "cross-providers-deps": []
+  },
+  "plexus": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "arrow>=0.16.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "postgres": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "psycopg2-binary>=2.7.4"
+    ],
+    "cross-providers-deps": [
+      "amazon"
+    ]
+  },
+  "presto": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "pandas>=0.17.1",
+      "presto-python-client>=0.8.2"
+    ],
+    "cross-providers-deps": [
+      "google",
+      "slack"
+    ]
+  },
+  "qubole": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "qds-sdk>=1.10.4"
+    ],
+    "cross-providers-deps": []
+  },
+  "redis": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "redis~=3.2"
+    ],
+    "cross-providers-deps": []
+  },
+  "salesforce": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "pandas>=0.17.1",
+      "simple-salesforce>=1.0.0",
+      "tableauserverclient"
+    ],
+    "cross-providers-deps": [
+      "tableau"
+    ]
+  },
+  "samba": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "smbprotocol>=1.5.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "segment": {
+    "deps": [
+      "analytics-python>=1.2.9",
+      "apache-airflow>=2.2.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "sendgrid": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "sendgrid>=6.0.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "sftp": {
+    "deps": [
+      "apache-airflow-providers-ssh>=2.1.0",
+      "apache-airflow>=2.2.0"
+    ],
+    "cross-providers-deps": [
+      "ssh"
+    ]
+  },
+  "singularity": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "spython>=0.0.56"
+    ],
+    "cross-providers-deps": []
+  },
+  "slack": {
+    "deps": [
+      "apache-airflow-providers-http",
+      "apache-airflow>=2.2.0",
+      "slack_sdk>=3.0.0"
+    ],
+    "cross-providers-deps": [
+      "http"
+    ]
+  },
+  "snowflake": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "snowflake-connector-python>=2.4.1",
+      "snowflake-sqlalchemy>=1.1.0"
+    ],
+    "cross-providers-deps": [
+      "slack"
+    ]
+  },
+  "sqlite": {
+    "deps": [],
+    "cross-providers-deps": []
+  },
+  "ssh": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "paramiko>=2.6.0",
+      "sshtunnel>=0.3.2"
+    ],
+    "cross-providers-deps": []
+  },
+  "tableau": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "tableauserverclient"
+    ],
+    "cross-providers-deps": []
+  },
+  "telegram": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "python-telegram-bot>=13.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "trino": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "pandas>=0.17.1",
+      "trino>=0.301.0"
+    ],
+    "cross-providers-deps": [
+      "google"
+    ]
+  },
+  "vertica": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "vertica-python>=0.5.1"
+    ],
+    "cross-providers-deps": []
+  },
+  "yandex": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "yandexcloud>=0.146.0"
+    ],
+    "cross-providers-deps": []
+  },
+  "zendesk": {
+    "deps": [
+      "apache-airflow>=2.2.0",
+      "zenpy>=2.0.24"
+    ],
+    "cross-providers-deps": []
+  }
+}
diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt
index dbf74a15b2..a6b6d5ad8e 100644
--- a/images/breeze/output-commands-hash.txt
+++ b/images/breeze/output-commands-hash.txt
@@ -28,7 +28,7 @@ self-upgrade:b5437c0a1a91533a11ee9d0a9692369c
 setup-autocomplete:355b72dee171c2fcba46fc90ac7c97b0
 shell:4680295fdd8a276d51518d29360c365c
 start-airflow:92cf775a952439a32d409cd2536da507
-static-checks:96a0bcc981be838ae32f53e804157ee5
+static-checks:c7adc5d6dff34624ef413d3d146ec974
 stop:8ebd8a42f1003495d37b884de5ac7ce6
 tests:ae8d62b505ff8f79bddc202fe9d575e3
 verify-image:a6b3c70957aea96a5d4d261f23359a2d
diff --git a/images/breeze/output-static-checks.svg b/images/breeze/output-static-checks.svg
index ae64d0be23..90272c90b4 100644
--- a/images/breeze/output-static-checks.svg
+++ b/images/breeze/output-static-checks.svg
@@ -1,4 +1,4 @@
-<svg class="rich-terminal" viewBox="0 0 1482 1270.0" xmlns="http://www.w3.org/2000/svg">
+<svg class="rich-terminal" viewBox="0 0 1482 1245.6" xmlns="http://www.w3.org/2000/svg">
     <!-- Generated with Rich https://www.textualize.io -->
     <style>
 
@@ -19,241 +19,237 @@
         font-weight: 700;
     }
 
-    .terminal-1143647577-matrix {
+    .terminal-618500324-matrix {
         font-family: Fira Code, monospace;
         font-size: 20px;
         line-height: 24.4px;
         font-variant-east-asian: full-width;
     }
 
-    .terminal-1143647577-title {
+    .terminal-618500324-title {
         font-size: 18px;
         font-weight: bold;
         font-family: arial;
     }
 
-    .terminal-1143647577-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-1143647577-r2 { fill: #c5c8c6 }
-.terminal-1143647577-r3 { fill: #d0b344;font-weight: bold }
-.terminal-1143647577-r4 { fill: #868887 }
-.terminal-1143647577-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-1143647577-r6 { fill: #98a84b;font-weight: bold }
-.terminal-1143647577-r7 { fill: #8d7b39 }
+    .terminal-618500324-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-618500324-r2 { fill: #c5c8c6 }
+.terminal-618500324-r3 { fill: #d0b344;font-weight: bold }
+.terminal-618500324-r4 { fill: #868887 }
+.terminal-618500324-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-618500324-r6 { fill: #98a84b;font-weight: bold }
+.terminal-618500324-r7 { fill: #8d7b39 }
     </style>
 
     <defs>
-    <clipPath id="terminal-1143647577-clip-terminal">
-      <rect x="0" y="0" width="1463.0" height="1219.0" />
+    <clipPath id="terminal-618500324-clip-terminal">
+      <rect x="0" y="0" width="1463.0" height="1194.6" />
     </clipPath>
-    <clipPath id="terminal-1143647577-line-0">
+    <clipPath id="terminal-618500324-line-0">
     <rect x="0" y="1.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-1">
+<clipPath id="terminal-618500324-line-1">
     <rect x="0" y="25.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-2">
+<clipPath id="terminal-618500324-line-2">
     <rect x="0" y="50.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-3">
+<clipPath id="terminal-618500324-line-3">
     <rect x="0" y="74.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-4">
+<clipPath id="terminal-618500324-line-4">
     <rect x="0" y="99.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-5">
+<clipPath id="terminal-618500324-line-5">
     <rect x="0" y="123.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-6">
+<clipPath id="terminal-618500324-line-6">
     <rect x="0" y="147.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-7">
+<clipPath id="terminal-618500324-line-7">
     <rect x="0" y="172.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-8">
+<clipPath id="terminal-618500324-line-8">
     <rect x="0" y="196.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-9">
+<clipPath id="terminal-618500324-line-9">
     <rect x="0" y="221.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-10">
+<clipPath id="terminal-618500324-line-10">
     <rect x="0" y="245.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-11">
+<clipPath id="terminal-618500324-line-11">
     <rect x="0" y="269.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-12">
+<clipPath id="terminal-618500324-line-12">
     <rect x="0" y="294.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-13">
+<clipPath id="terminal-618500324-line-13">
     <rect x="0" y="318.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-14">
+<clipPath id="terminal-618500324-line-14">
     <rect x="0" y="343.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-15">
+<clipPath id="terminal-618500324-line-15">
     <rect x="0" y="367.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-16">
+<clipPath id="terminal-618500324-line-16">
     <rect x="0" y="391.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-17">
+<clipPath id="terminal-618500324-line-17">
     <rect x="0" y="416.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-18">
+<clipPath id="terminal-618500324-line-18">
     <rect x="0" y="440.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-19">
+<clipPath id="terminal-618500324-line-19">
     <rect x="0" y="465.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-20">
+<clipPath id="terminal-618500324-line-20">
     <rect x="0" y="489.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-21">
+<clipPath id="terminal-618500324-line-21">
     <rect x="0" y="513.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-22">
+<clipPath id="terminal-618500324-line-22">
     <rect x="0" y="538.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-23">
+<clipPath id="terminal-618500324-line-23">
     <rect x="0" y="562.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-24">
+<clipPath id="terminal-618500324-line-24">
     <rect x="0" y="587.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-25">
+<clipPath id="terminal-618500324-line-25">
     <rect x="0" y="611.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-26">
+<clipPath id="terminal-618500324-line-26">
     <rect x="0" y="635.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-27">
+<clipPath id="terminal-618500324-line-27">
     <rect x="0" y="660.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-28">
+<clipPath id="terminal-618500324-line-28">
     <rect x="0" y="684.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-29">
+<clipPath id="terminal-618500324-line-29">
     <rect x="0" y="709.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-30">
+<clipPath id="terminal-618500324-line-30">
     <rect x="0" y="733.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-31">
+<clipPath id="terminal-618500324-line-31">
     <rect x="0" y="757.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-32">
+<clipPath id="terminal-618500324-line-32">
     <rect x="0" y="782.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-33">
+<clipPath id="terminal-618500324-line-33">
     <rect x="0" y="806.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-34">
+<clipPath id="terminal-618500324-line-34">
     <rect x="0" y="831.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-35">
+<clipPath id="terminal-618500324-line-35">
     <rect x="0" y="855.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-36">
+<clipPath id="terminal-618500324-line-36">
     <rect x="0" y="879.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-37">
+<clipPath id="terminal-618500324-line-37">
     <rect x="0" y="904.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-38">
+<clipPath id="terminal-618500324-line-38">
     <rect x="0" y="928.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-39">
+<clipPath id="terminal-618500324-line-39">
     <rect x="0" y="953.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-40">
+<clipPath id="terminal-618500324-line-40">
     <rect x="0" y="977.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-41">
+<clipPath id="terminal-618500324-line-41">
     <rect x="0" y="1001.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-42">
+<clipPath id="terminal-618500324-line-42">
     <rect x="0" y="1026.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-43">
+<clipPath id="terminal-618500324-line-43">
     <rect x="0" y="1050.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-44">
+<clipPath id="terminal-618500324-line-44">
     <rect x="0" y="1075.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-45">
+<clipPath id="terminal-618500324-line-45">
     <rect x="0" y="1099.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-46">
+<clipPath id="terminal-618500324-line-46">
     <rect x="0" y="1123.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-47">
+<clipPath id="terminal-618500324-line-47">
     <rect x="0" y="1148.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1143647577-line-48">
-    <rect x="0" y="1172.7" width="1464" height="24.65"/>
-            </clipPath>
     </defs>
 
-    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="1268" rx="8"/><text class="terminal-1143647577-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;static-checks</text>
+    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="1243.6" rx="8"/><text class="terminal-618500324-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;static-checks</text>
             <g transform="translate(26,22)">
             <circle cx="0" cy="0" r="7" fill="#ff5f57"/>
             <circle cx="22" cy="0" r="7" fill="#febc2e"/>
             <circle cx="44" cy="0" r="7" fill="#28c840"/>
             </g>
         
-    <g transform="translate(9, 41)" clip-path="url(#terminal-1143647577-clip-terminal)">
+    <g transform="translate(9, 41)" clip-path="url(#terminal-618500324-clip-terminal)">
     
-    <g class="terminal-1143647577-matrix">
-    <text class="terminal-1143647577-r2" x="1464" y="20" textLength="12.2" clip-path="url(#terminal-1143647577-line-0)">
-</text><text class="terminal-1143647577-r3" x="12.2" y="44.4" textLength="85.4" clip-path="url(#terminal-1143647577-line-1)">Usage:&#160;</text><text class="terminal-1143647577-r1" x="97.6" y="44.4" textLength="610" clip-path="url(#terminal-1143647577-line-1)">breeze&#160;static-checks&#160;[OPTIONS]&#160;[PRECOMMIT_ARGS]...</text><text class="terminal-1143647577-r2" x="1464" y="44.4" textLength="12.2" clip-path="url(#terminal-1143647577-line-1)">
-</text><text class="terminal-1143647577-r2" x="1464" y="68.8" textLength="12.2" clip-path="url(#terminal-1143647577-line-2)">
-</text><text class="terminal-1143647577-r2" x="12.2" y="93.2" textLength="219.6" clip-path="url(#terminal-1143647577-line-3)">Run&#160;static&#160;checks.</text><text class="terminal-1143647577-r2" x="1464" y="93.2" textLength="12.2" clip-path="url(#terminal-1143647577-line-3)">
-</text><text class="terminal-1143647577-r2" x="1464" y="117.6" textLength="12.2" clip-path="url(#terminal-1143647577-line-4)">
-</text><text class="terminal-1143647577-r4" x="0" y="142" textLength="24.4" clip-path="url(#terminal-1143647577-line-5)">╭─</text><text class="terminal-1143647577-r4" x="24.4" y="142" textLength="1415.2" clip-path="url(#terminal-1143647577-line-5)">&#160;Pre-commit&#160;flags&#160;──────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-1143647577-r4" x="1439.6" y="142" textLength="24.4" clip-path="url(#terminal-114364 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="166.4" textLength="12.2" clip-path="url(#terminal-1143647577-line-6)">│</text><text class="terminal-1143647577-r5" x="24.4" y="166.4" textLength="12.2" clip-path="url(#terminal-1143647577-line-6)">-</text><text class="terminal-1143647577-r5" x="36.6" y="166.4" textLength="61" clip-path="url(#terminal-1143647577-line-6)">-type</text><text class="terminal-1143647577-r6" x="317.2" y="166.4" textLength="24.4" clip-path="url(#terminal-11436 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="190.8" textLength="12.2" clip-path="url(#terminal-1143647577-line-7)">│</text><text class="terminal-1143647577-r7" x="366" y="190.8" textLength="1073.6" clip-path="url(#terminal-1143647577-line-7)">(all&#160;|&#160;black&#160;|&#160;blacken-docs&#160;|&#160;check-airflow-2-2-compatibility&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="215.2" textLength="12.2" clip-path="url(#terminal-1143647577-line-8)">│</text><text class="terminal-1143647577-r7" x="366" y="215.2" textLength="1073.6" clip-path="url(#terminal-1143647577-line-8)">check-airflow-config-yaml-consistent&#160;|&#160;check-airflow-providers-have-extras&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1143647577-r4" x="1451.8" y="215.2" textLength="1 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="239.6" textLength="12.2" clip-path="url(#terminal-1143647577-line-9)">│</text><text class="terminal-1143647577-r7" x="366" y="239.6" textLength="1073.6" clip-path="url(#terminal-1143647577-line-9)">check-apache-license-rat&#160;|&#160;check-base-operator-partial-arguments&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal [...]
-</text><text class="terminal-1143647577-r4" x="0" y="264" textLength="12.2" clip-path="url(#terminal-1143647577-line-10)">│</text><text class="terminal-1143647577-r7" x="366" y="264" textLength="1073.6" clip-path="url(#terminal-1143647577-line-10)">check-base-operator-usage&#160;|&#160;check-boring-cyborg-configuration&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text cl [...]
-</text><text class="terminal-1143647577-r4" x="0" y="288.4" textLength="12.2" clip-path="url(#terminal-1143647577-line-11)">│</text><text class="terminal-1143647577-r7" x="366" y="288.4" textLength="1073.6" clip-path="url(#terminal-1143647577-line-11)">check-breeze-top-dependencies-limited&#160;|&#160;check-builtin-literals&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text cla [...]
-</text><text class="terminal-1143647577-r4" x="0" y="312.8" textLength="12.2" clip-path="url(#terminal-1143647577-line-12)">│</text><text class="terminal-1143647577-r7" x="366" y="312.8" textLength="1073.6" clip-path="url(#terminal-1143647577-line-12)">check-changelog-has-no-duplicates&#160;|&#160;check-daysago-import-from-utils&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1143647577-r [...]
-</text><text class="terminal-1143647577-r4" x="0" y="337.2" textLength="12.2" clip-path="url(#terminal-1143647577-line-13)">│</text><text class="terminal-1143647577-r7" x="366" y="337.2" textLength="1073.6" clip-path="url(#terminal-1143647577-line-13)">check-docstring-param-types&#160;|&#160;check-example-dags-urls&#160;|&#160;check-executables-have-shebangs&#160;</text><text class="terminal-1143647577-r4" x="1451.8" y="337.2" textLength="12.2" clip-path="url(#terminal-1143647577-line-13 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="361.6" textLength="12.2" clip-path="url(#terminal-1143647577-line-14)">│</text><text class="terminal-1143647577-r7" x="366" y="361.6" textLength="1073.6" clip-path="url(#terminal-1143647577-line-14)">|&#160;check-extra-packages-references&#160;|&#160;check-extras-order&#160;|&#160;check-for-inclusive-language&#160;|&#160;</text><text class="terminal-1143647577-r4" x="1451.8" y="361.6" textLength="12.2" clip-path="url(#terminal-11436475 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="386" textLength="12.2" clip-path="url(#terminal-1143647577-line-15)">│</text><text class="terminal-1143647577-r7" x="366" y="386" textLength="1073.6" clip-path="url(#terminal-1143647577-line-15)">check-hooks-apply&#160;|&#160;check-incorrect-use-of-LoggingMixin&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="410.4" textLength="12.2" clip-path="url(#terminal-1143647577-line-16)">│</text><text class="terminal-1143647577-r7" x="366" y="410.4" textLength="1073.6" clip-path="url(#terminal-1143647577-line-16)">check-integrations-are-consistent&#160;|&#160;check-merge-conflict&#160;|&#160;check-newsfragments-are-valid</text><text class="terminal-1143647577-r4" x="1451.8" y="410.4" textLength="12.2" clip-path="url(#terminal-1143647577-line-16)">│< [...]
-</text><text class="terminal-1143647577-r4" x="0" y="434.8" textLength="12.2" clip-path="url(#terminal-1143647577-line-17)">│</text><text class="terminal-1143647577-r7" x="366" y="434.8" textLength="1073.6" clip-path="url(#terminal-1143647577-line-17)">|&#160;check-no-providers-in-core-examples&#160;|&#160;check-no-relative-imports&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="termin [...]
-</text><text class="terminal-1143647577-r4" x="0" y="459.2" textLength="12.2" clip-path="url(#terminal-1143647577-line-18)">│</text><text class="terminal-1143647577-r7" x="366" y="459.2" textLength="1073.6" clip-path="url(#terminal-1143647577-line-18)">check-persist-credentials-disabled-in-github-workflows&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#16 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="483.6" textLength="12.2" clip-path="url(#terminal-1143647577-line-19)">│</text><text class="terminal-1143647577-r7" x="366" y="483.6" textLength="1073.6" clip-path="url(#terminal-1143647577-line-19)">check-pre-commit-information-consistent&#160;|&#160;check-provide-create-sessions-imports&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1143647577-r4" x="1451.8" y="483.6" textLength="12.2" clip-path="url(#te [...]
-</text><text class="terminal-1143647577-r4" x="0" y="508" textLength="12.2" clip-path="url(#terminal-1143647577-line-20)">│</text><text class="terminal-1143647577-r7" x="366" y="508" textLength="1073.6" clip-path="url(#terminal-1143647577-line-20)">check-provider-yaml-valid&#160;|&#160;check-providers-init-file-missing&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text cl [...]
-</text><text class="terminal-1143647577-r4" x="0" y="532.4" textLength="12.2" clip-path="url(#terminal-1143647577-line-21)">│</text><text class="terminal-1143647577-r7" x="366" y="532.4" textLength="1073.6" clip-path="url(#terminal-1143647577-line-21)">check-providers-subpackages-init-file-exist&#160;|&#160;check-pydevd-left-in-code&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1143647577-r4" x="1451.8" y="532 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="556.8" textLength="12.2" clip-path="url(#terminal-1143647577-line-22)">│</text><text class="terminal-1143647577-r7" x="366" y="556.8" textLength="1073.6" clip-path="url(#terminal-1143647577-line-22)">check-revision-heads-map&#160;|&#160;check-safe-filter-usage-in-html&#160;|&#160;check-setup-order&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1143647577-r4" x="1451.8" y="556.8" textLength="12.2" cli [...]
-</text><text class="terminal-1143647577-r4" x="0" y="581.2" textLength="12.2" clip-path="url(#terminal-1143647577-line-23)">│</text><text class="terminal-1143647577-r7" x="366" y="581.2" textLength="1073.6" clip-path="url(#terminal-1143647577-line-23)">check-start-date-not-used-in-defaults&#160;|&#160;check-system-tests-present&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1143647 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="605.6" textLength="12.2" clip-path="url(#terminal-1143647577-line-24)">│</text><text class="terminal-1143647577-r7" x="366" y="605.6" textLength="1073.6" clip-path="url(#terminal-1143647577-line-24)">check-system-tests-tocs&#160;|&#160;check-xml&#160;|&#160;codespell&#160;|&#160;create-missing-init-py-files-tests&#160;|&#160;&#160;</text><text class="terminal-1143647577-r4" x="1451.8" y="605.6" textLength="12.2" clip-path="url(#termina [...]
-</text><text class="terminal-1143647577-r4" x="0" y="630" textLength="12.2" clip-path="url(#terminal-1143647577-line-25)">│</text><text class="terminal-1143647577-r7" x="366" y="630" textLength="1073.6" clip-path="url(#terminal-1143647577-line-25)">debug-statements&#160;|&#160;detect-private-key&#160;|&#160;doctoc&#160;|&#160;end-of-file-fixer&#160;|&#160;fix-encoding-pragma</text><text class="terminal-1143647577-r4" x="1451.8" y="630" textLength="12.2" clip-path="url(#terminal-114364757 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="654.4" textLength="12.2" clip-path="url(#terminal-1143647577-line-26)">│</text><text class="terminal-1143647577-r7" x="366" y="654.4" textLength="1073.6" clip-path="url(#terminal-1143647577-line-26)">|&#160;flynt&#160;|&#160;forbid-tabs&#160;|&#160;identity&#160;|&#160;insert-license&#160;|&#160;isort&#160;|&#160;lint-chart-schema&#160;|&#160;lint-css</text><text class="terminal-1143647577-r4" x="1451.8" y="654.4" textLength="12.2" cli [...]
-</text><text class="terminal-1143647577-r4" x="0" y="678.8" textLength="12.2" clip-path="url(#terminal-1143647577-line-27)">│</text><text class="terminal-1143647577-r7" x="366" y="678.8" textLength="1073.6" clip-path="url(#terminal-1143647577-line-27)">|&#160;lint-dockerfile&#160;|&#160;lint-helm-chart&#160;|&#160;lint-javascript&#160;|&#160;lint-json-schema&#160;|&#160;lint-markdown</text><text class="terminal-1143647577-r4" x="1451.8" y="678.8" textLength="12.2" clip-path="url(#termina [...]
-</text><text class="terminal-1143647577-r4" x="0" y="703.2" textLength="12.2" clip-path="url(#terminal-1143647577-line-28)">│</text><text class="terminal-1143647577-r7" x="366" y="703.2" textLength="1073.6" clip-path="url(#terminal-1143647577-line-28)">|&#160;lint-openapi&#160;|&#160;mixed-line-ending&#160;|&#160;pretty-format-json&#160;|&#160;pydocstyle&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="t [...]
-</text><text class="terminal-1143647577-r4" x="0" y="727.6" textLength="12.2" clip-path="url(#terminal-1143647577-line-29)">│</text><text class="terminal-1143647577-r7" x="366" y="727.6" textLength="1073.6" clip-path="url(#terminal-1143647577-line-29)">python-no-log-warn&#160;|&#160;pyupgrade&#160;|&#160;rst-backticks&#160;|&#160;run-flake8&#160;|&#160;run-mypy&#160;|&#160;run-shellcheck&#160;</text><text class="terminal-1143647577-r4" x="1451.8" y="727.6" textLength="12.2" clip-path="ur [...]
-</text><text class="terminal-1143647577-r4" x="0" y="752" textLength="12.2" clip-path="url(#terminal-1143647577-line-30)">│</text><text class="terminal-1143647577-r7" x="366" y="752" textLength="1073.6" clip-path="url(#terminal-1143647577-line-30)">|&#160;static-check-autoflake&#160;|&#160;trailing-whitespace&#160;|&#160;update-breeze-cmd-output&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1143647577-r4" x="1451.8" y="75 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="776.4" textLength="12.2" clip-path="url(#terminal-1143647577-line-31)">│</text><text class="terminal-1143647577-r7" x="366" y="776.4" textLength="1073.6" clip-path="url(#terminal-1143647577-line-31)">update-breeze-readme-config-hash&#160;|&#160;update-extras&#160;|&#160;update-in-the-wild-to-be-sorted&#160;|&#160;&#160;&#160;&#160;</text><text class="terminal-1143647577-r4" x="1451.8" y="776.4" textLength="12.2" clip-path="url(#termina [...]
-</text><text class="terminal-1143647577-r4" x="0" y="800.8" textLength="12.2" clip-path="url(#terminal-1143647577-line-32)">│</text><text class="terminal-1143647577-r7" x="366" y="800.8" textLength="1073.6" clip-path="url(#terminal-1143647577-line-32)">update-inlined-dockerfile-scripts&#160;|&#160;update-local-yml-file&#160;|&#160;update-migration-references&#160;</text><text class="terminal-1143647577-r4" x="1451.8" y="800.8" textLength="12.2" clip-path="url(#terminal-1143647577-line-32 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="825.2" textLength="12.2" clip-path="url(#terminal-1143647577-line-33)">│</text><text class="terminal-1143647577-r7" x="366" y="825.2" textLength="1073.6" clip-path="url(#terminal-1143647577-line-33)">|&#160;update-providers-dependencies&#160;|&#160;update-setup-cfg-file&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160; [...]
-</text><text class="terminal-1143647577-r4" x="0" y="849.6" textLength="12.2" clip-path="url(#terminal-1143647577-line-34)">│</text><text class="terminal-1143647577-r7" x="366" y="849.6" textLength="1073.6" clip-path="url(#terminal-1143647577-line-34)">update-spelling-wordlist-to-be-sorted&#160;|&#160;update-supported-versions&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-11 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="874" textLength="12.2" clip-path="url(#terminal-1143647577-line-35)">│</text><text class="terminal-1143647577-r7" x="366" y="874" textLength="1073.6" clip-path="url(#terminal-1143647577-line-35)">update-vendored-in-k8s-json-schema&#160;|&#160;update-version&#160;|&#160;yamllint&#160;|&#160;yesqa)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1143647577- [...]
-</text><text class="terminal-1143647577-r4" x="0" y="898.4" textLength="12.2" clip-path="url(#terminal-1143647577-line-36)">│</text><text class="terminal-1143647577-r5" x="24.4" y="898.4" textLength="12.2" clip-path="url(#terminal-1143647577-line-36)">-</text><text class="terminal-1143647577-r5" x="36.6" y="898.4" textLength="61" clip-path="url(#terminal-1143647577-line-36)">-file</text><text class="terminal-1143647577-r6" x="317.2" y="898.4" textLength="24.4" clip-path="url(#terminal-11 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="922.8" textLength="12.2" clip-path="url(#terminal-1143647577-line-37)">│</text><text class="terminal-1143647577-r5" x="24.4" y="922.8" textLength="12.2" clip-path="url(#terminal-1143647577-line-37)">-</text><text class="terminal-1143647577-r5" x="36.6" y="922.8" textLength="48.8" clip-path="url(#terminal-1143647577-line-37)">-all</text><text class="terminal-1143647577-r5" x="85.4" y="922.8" textLength="73.2" clip-path="url(#terminal-11 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="947.2" textLength="12.2" clip-path="url(#terminal-1143647577-line-38)">│</text><text class="terminal-1143647577-r5" x="24.4" y="947.2" textLength="12.2" clip-path="url(#terminal-1143647577-line-38)">-</text><text class="terminal-1143647577-r5" x="36.6" y="947.2" textLength="61" clip-path="url(#terminal-1143647577-line-38)">-show</text><text class="terminal-1143647577-r5" x="97.6" y="947.2" textLength="195.2" clip-path="url(#terminal-11 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="971.6" textLength="12.2" clip-path="url(#terminal-1143647577-line-39)">│</text><text class="terminal-1143647577-r5" x="24.4" y="971.6" textLength="12.2" clip-path="url(#terminal-1143647577-line-39)">-</text><text class="terminal-1143647577-r5" x="36.6" y="971.6" textLength="61" clip-path="url(#terminal-1143647577-line-39)">-last</text><text class="terminal-1143647577-r5" x="97.6" y="971.6" textLength="85.4" clip-path="url(#terminal-114 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="996" textLength="1464" clip-path="url(#terminal-1143647577-line-40)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1143647577-r2" x="1464" y="996" textLength="12.2" clip-path="url(#terminal-1143647577-line-40)">
-</text><text class="terminal-1143647577-r4" x="0" y="1020.4" textLength="24.4" clip-path="url(#terminal-1143647577-line-41)">╭─</text><text class="terminal-1143647577-r4" x="24.4" y="1020.4" textLength="1415.2" clip-path="url(#terminal-1143647577-line-41)">&#160;Options&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-1143647577-r4" x="1439.6" y="1020.4" textLength="24.4" clip-path="url(#terminal- [...]
-</text><text class="terminal-1143647577-r4" x="0" y="1044.8" textLength="12.2" clip-path="url(#terminal-1143647577-line-42)">│</text><text class="terminal-1143647577-r5" x="24.4" y="1044.8" textLength="12.2" clip-path="url(#terminal-1143647577-line-42)">-</text><text class="terminal-1143647577-r5" x="36.6" y="1044.8" textLength="85.4" clip-path="url(#terminal-1143647577-line-42)">-commit</text><text class="terminal-1143647577-r5" x="122" y="1044.8" textLength="48.8" clip-path="url(#termi [...]
-</text><text class="terminal-1143647577-r4" x="0" y="1069.2" textLength="12.2" clip-path="url(#terminal-1143647577-line-43)">│</text><text class="terminal-1143647577-r2" x="329.4" y="1069.2" textLength="183" clip-path="url(#terminal-1143647577-line-43)">exclusive&#160;with&#160;</text><text class="terminal-1143647577-r5" x="512.4" y="1069.2" textLength="12.2" clip-path="url(#terminal-1143647577-line-43)">-</text><text class="terminal-1143647577-r5" x="524.6" y="1069.2" textLength="61" cl [...]
-</text><text class="terminal-1143647577-r4" x="0" y="1093.6" textLength="12.2" clip-path="url(#terminal-1143647577-line-44)">│</text><text class="terminal-1143647577-r7" x="329.4" y="1093.6" textLength="1110.2" clip-path="url(#terminal-1143647577-line-44)">(TEXT)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
-</text><text class="terminal-1143647577-r4" x="0" y="1118" textLength="12.2" clip-path="url(#terminal-1143647577-line-45)">│</text><text class="terminal-1143647577-r5" x="24.4" y="1118" textLength="12.2" clip-path="url(#terminal-1143647577-line-45)">-</text><text class="terminal-1143647577-r5" x="36.6" y="1118" textLength="97.6" clip-path="url(#terminal-1143647577-line-45)">-verbose</text><text class="terminal-1143647577-r6" x="280.6" y="1118" textLength="24.4" clip-path="url(#terminal-1 [...]
-</text><text class="terminal-1143647577-r4" x="0" y="1142.4" textLength="12.2" clip-path="url(#terminal-1143647577-line-46)">│</text><text class="terminal-1143647577-r5" x="24.4" y="1142.4" textLength="12.2" clip-path="url(#terminal-1143647577-line-46)">-</text><text class="terminal-1143647577-r5" x="36.6" y="1142.4" textLength="48.8" clip-path="url(#terminal-1143647577-line-46)">-dry</text><text class="terminal-1143647577-r5" x="85.4" y="1142.4" textLength="48.8" clip-path="url(#termina [...]
-</text><text class="terminal-1143647577-r4" x="0" y="1166.8" textLength="12.2" clip-path="url(#terminal-1143647577-line-47)">│</text><text class="terminal-1143647577-r5" x="24.4" y="1166.8" textLength="12.2" clip-path="url(#terminal-1143647577-line-47)">-</text><text class="terminal-1143647577-r5" x="36.6" y="1166.8" textLength="85.4" clip-path="url(#terminal-1143647577-line-47)">-github</text><text class="terminal-1143647577-r5" x="122" y="1166.8" textLength="134.2" clip-path="url(#term [...]
-</text><text class="terminal-1143647577-r4" x="0" y="1191.2" textLength="12.2" clip-path="url(#terminal-1143647577-line-48)">│</text><text class="terminal-1143647577-r5" x="24.4" y="1191.2" textLength="12.2" clip-path="url(#terminal-1143647577-line-48)">-</text><text class="terminal-1143647577-r5" x="36.6" y="1191.2" textLength="61" clip-path="url(#terminal-1143647577-line-48)">-help</text><text class="terminal-1143647577-r6" x="280.6" y="1191.2" textLength="24.4" clip-path="url(#termina [...]
-</text><text class="terminal-1143647577-r4" x="0" y="1215.6" textLength="1464" clip-path="url(#terminal-1143647577-line-49)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1143647577-r2" x="1464" y="1215.6" textLength="12.2" clip-path="url(#terminal-1143647577-line-49)">
+    <g class="terminal-618500324-matrix">
+    <text class="terminal-618500324-r2" x="1464" y="20" textLength="12.2" clip-path="url(#terminal-618500324-line-0)">
+</text><text class="terminal-618500324-r3" x="12.2" y="44.4" textLength="85.4" clip-path="url(#terminal-618500324-line-1)">Usage:&#160;</text><text class="terminal-618500324-r1" x="97.6" y="44.4" textLength="610" clip-path="url(#terminal-618500324-line-1)">breeze&#160;static-checks&#160;[OPTIONS]&#160;[PRECOMMIT_ARGS]...</text><text class="terminal-618500324-r2" x="1464" y="44.4" textLength="12.2" clip-path="url(#terminal-618500324-line-1)">
+</text><text class="terminal-618500324-r2" x="1464" y="68.8" textLength="12.2" clip-path="url(#terminal-618500324-line-2)">
+</text><text class="terminal-618500324-r2" x="12.2" y="93.2" textLength="219.6" clip-path="url(#terminal-618500324-line-3)">Run&#160;static&#160;checks.</text><text class="terminal-618500324-r2" x="1464" y="93.2" textLength="12.2" clip-path="url(#terminal-618500324-line-3)">
+</text><text class="terminal-618500324-r2" x="1464" y="117.6" textLength="12.2" clip-path="url(#terminal-618500324-line-4)">
+</text><text class="terminal-618500324-r4" x="0" y="142" textLength="24.4" clip-path="url(#terminal-618500324-line-5)">╭─</text><text class="terminal-618500324-r4" x="24.4" y="142" textLength="1415.2" clip-path="url(#terminal-618500324-line-5)">&#160;Pre-commit&#160;flags&#160;──────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-618500324-r4" x="1439.6" y="142" textLength="24.4" clip-path="url(#terminal-618500324-l [...]
+</text><text class="terminal-618500324-r4" x="0" y="166.4" textLength="12.2" clip-path="url(#terminal-618500324-line-6)">│</text><text class="terminal-618500324-r5" x="24.4" y="166.4" textLength="12.2" clip-path="url(#terminal-618500324-line-6)">-</text><text class="terminal-618500324-r5" x="36.6" y="166.4" textLength="61" clip-path="url(#terminal-618500324-line-6)">-type</text><text class="terminal-618500324-r6" x="317.2" y="166.4" textLength="24.4" clip-path="url(#terminal-618500324-li [...]
+</text><text class="terminal-618500324-r4" x="0" y="190.8" textLength="12.2" clip-path="url(#terminal-618500324-line-7)">│</text><text class="terminal-618500324-r7" x="366" y="190.8" textLength="1073.6" clip-path="url(#terminal-618500324-line-7)">(all&#160;|&#160;black&#160;|&#160;blacken-docs&#160;|&#160;check-airflow-2-2-compatibility&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
+</text><text class="terminal-618500324-r4" x="0" y="215.2" textLength="12.2" clip-path="url(#terminal-618500324-line-8)">│</text><text class="terminal-618500324-r7" x="366" y="215.2" textLength="1073.6" clip-path="url(#terminal-618500324-line-8)">check-airflow-config-yaml-consistent&#160;|&#160;check-apache-license-rat&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="termina [...]
+</text><text class="terminal-618500324-r4" x="0" y="239.6" textLength="12.2" clip-path="url(#terminal-618500324-line-9)">│</text><text class="terminal-618500324-r7" x="366" y="239.6" textLength="1073.6" clip-path="url(#terminal-618500324-line-9)">check-base-operator-partial-arguments&#160;|&#160;check-base-operator-usage&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-61850032 [...]
+</text><text class="terminal-618500324-r4" x="0" y="264" textLength="12.2" clip-path="url(#terminal-618500324-line-10)">│</text><text class="terminal-618500324-r7" x="366" y="264" textLength="1073.6" clip-path="url(#terminal-618500324-line-10)">check-boring-cyborg-configuration&#160;|&#160;check-breeze-top-dependencies-limited&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-618500324-r4" x="1451.8" y="264" textLength="12.2" [...]
+</text><text class="terminal-618500324-r4" x="0" y="288.4" textLength="12.2" clip-path="url(#terminal-618500324-line-11)">│</text><text class="terminal-618500324-r7" x="366" y="288.4" textLength="1073.6" clip-path="url(#terminal-618500324-line-11)">check-builtin-literals&#160;|&#160;check-changelog-has-no-duplicates&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160; [...]
+</text><text class="terminal-618500324-r4" x="0" y="312.8" textLength="12.2" clip-path="url(#terminal-618500324-line-12)">│</text><text class="terminal-618500324-r7" x="366" y="312.8" textLength="1073.6" clip-path="url(#terminal-618500324-line-12)">check-daysago-import-from-utils&#160;|&#160;check-docstring-param-types&#160;|&#160;check-example-dags-urls&#160;</text><text class="terminal-618500324-r4" x="1451.8" y="312.8" textLength="12.2" clip-path="url(#terminal-618500324-line-12)">│</ [...]
+</text><text class="terminal-618500324-r4" x="0" y="337.2" textLength="12.2" clip-path="url(#terminal-618500324-line-13)">│</text><text class="terminal-618500324-r7" x="366" y="337.2" textLength="1073.6" clip-path="url(#terminal-618500324-line-13)">|&#160;check-executables-have-shebangs&#160;|&#160;check-extra-packages-references&#160;|&#160;check-extras-order</text><text class="terminal-618500324-r4" x="1451.8" y="337.2" textLength="12.2" clip-path="url(#terminal-618500324-line-13)">│</ [...]
+</text><text class="terminal-618500324-r4" x="0" y="361.6" textLength="12.2" clip-path="url(#terminal-618500324-line-14)">│</text><text class="terminal-618500324-r7" x="366" y="361.6" textLength="1073.6" clip-path="url(#terminal-618500324-line-14)">|&#160;check-for-inclusive-language&#160;|&#160;check-hooks-apply&#160;|&#160;check-incorrect-use-of-LoggingMixin</text><text class="terminal-618500324-r4" x="1451.8" y="361.6" textLength="12.2" clip-path="url(#terminal-618500324-line-14)">│</ [...]
+</text><text class="terminal-618500324-r4" x="0" y="386" textLength="12.2" clip-path="url(#terminal-618500324-line-15)">│</text><text class="terminal-618500324-r7" x="366" y="386" textLength="1073.6" clip-path="url(#terminal-618500324-line-15)">|&#160;check-integrations-are-consistent&#160;|&#160;check-merge-conflict&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
+</text><text class="terminal-618500324-r4" x="0" y="410.4" textLength="12.2" clip-path="url(#terminal-618500324-line-16)">│</text><text class="terminal-618500324-r7" x="366" y="410.4" textLength="1073.6" clip-path="url(#terminal-618500324-line-16)">check-newsfragments-are-valid&#160;|&#160;check-no-providers-in-core-examples&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-618500324-r4" x= [...]
+</text><text class="terminal-618500324-r4" x="0" y="434.8" textLength="12.2" clip-path="url(#terminal-618500324-line-17)">│</text><text class="terminal-618500324-r7" x="366" y="434.8" textLength="1073.6" clip-path="url(#terminal-618500324-line-17)">check-no-relative-imports&#160;|&#160;check-persist-credentials-disabled-in-github-workflows&#160;|&#160;&#160;&#160;&#160;</text><text class="terminal-618500324-r4" x="1451.8" y="434.8" textLength="12.2" clip-path="url(#terminal-618500324-lin [...]
+</text><text class="terminal-618500324-r4" x="0" y="459.2" textLength="12.2" clip-path="url(#terminal-618500324-line-18)">│</text><text class="terminal-618500324-r7" x="366" y="459.2" textLength="1073.6" clip-path="url(#terminal-618500324-line-18)">check-pre-commit-information-consistent&#160;|&#160;check-provide-create-sessions-imports&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-618500324-r4" x="1451.8" y="459.2" textLength="12.2" clip-path="url(#termina [...]
+</text><text class="terminal-618500324-r4" x="0" y="483.6" textLength="12.2" clip-path="url(#terminal-618500324-line-19)">│</text><text class="terminal-618500324-r7" x="366" y="483.6" textLength="1073.6" clip-path="url(#terminal-618500324-line-19)">check-provider-yaml-valid&#160;|&#160;check-providers-init-file-missing&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text cl [...]
+</text><text class="terminal-618500324-r4" x="0" y="508" textLength="12.2" clip-path="url(#terminal-618500324-line-20)">│</text><text class="terminal-618500324-r7" x="366" y="508" textLength="1073.6" clip-path="url(#terminal-618500324-line-20)">check-providers-subpackages-init-file-exist&#160;|&#160;check-pydevd-left-in-code&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-618500324-r4" x="1451.8" y="508" textLen [...]
+</text><text class="terminal-618500324-r4" x="0" y="532.4" textLength="12.2" clip-path="url(#terminal-618500324-line-21)">│</text><text class="terminal-618500324-r7" x="366" y="532.4" textLength="1073.6" clip-path="url(#terminal-618500324-line-21)">check-revision-heads-map&#160;|&#160;check-safe-filter-usage-in-html&#160;|&#160;check-setup-order&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-618500324-r4" x="1451.8" y="532.4" textLength="12.2" clip-pat [...]
+</text><text class="terminal-618500324-r4" x="0" y="556.8" textLength="12.2" clip-path="url(#terminal-618500324-line-22)">│</text><text class="terminal-618500324-r7" x="366" y="556.8" textLength="1073.6" clip-path="url(#terminal-618500324-line-22)">check-start-date-not-used-in-defaults&#160;|&#160;check-system-tests-present&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-618500324-r [...]
+</text><text class="terminal-618500324-r4" x="0" y="581.2" textLength="12.2" clip-path="url(#terminal-618500324-line-23)">│</text><text class="terminal-618500324-r7" x="366" y="581.2" textLength="1073.6" clip-path="url(#terminal-618500324-line-23)">check-system-tests-tocs&#160;|&#160;check-xml&#160;|&#160;codespell&#160;|&#160;create-missing-init-py-files-tests&#160;|&#160;&#160;</text><text class="terminal-618500324-r4" x="1451.8" y="581.2" textLength="12.2" clip-path="url(#terminal-618 [...]
+</text><text class="terminal-618500324-r4" x="0" y="605.6" textLength="12.2" clip-path="url(#terminal-618500324-line-24)">│</text><text class="terminal-618500324-r7" x="366" y="605.6" textLength="1073.6" clip-path="url(#terminal-618500324-line-24)">debug-statements&#160;|&#160;detect-private-key&#160;|&#160;doctoc&#160;|&#160;end-of-file-fixer&#160;|&#160;fix-encoding-pragma</text><text class="terminal-618500324-r4" x="1451.8" y="605.6" textLength="12.2" clip-path="url(#terminal-61850032 [...]
+</text><text class="terminal-618500324-r4" x="0" y="630" textLength="12.2" clip-path="url(#terminal-618500324-line-25)">│</text><text class="terminal-618500324-r7" x="366" y="630" textLength="1073.6" clip-path="url(#terminal-618500324-line-25)">|&#160;flynt&#160;|&#160;forbid-tabs&#160;|&#160;identity&#160;|&#160;insert-license&#160;|&#160;isort&#160;|&#160;lint-chart-schema&#160;|&#160;lint-css</text><text class="terminal-618500324-r4" x="1451.8" y="630" textLength="12.2" clip-path="url [...]
+</text><text class="terminal-618500324-r4" x="0" y="654.4" textLength="12.2" clip-path="url(#terminal-618500324-line-26)">│</text><text class="terminal-618500324-r7" x="366" y="654.4" textLength="1073.6" clip-path="url(#terminal-618500324-line-26)">|&#160;lint-dockerfile&#160;|&#160;lint-helm-chart&#160;|&#160;lint-javascript&#160;|&#160;lint-json-schema&#160;|&#160;lint-markdown</text><text class="terminal-618500324-r4" x="1451.8" y="654.4" textLength="12.2" clip-path="url(#terminal-618 [...]
+</text><text class="terminal-618500324-r4" x="0" y="678.8" textLength="12.2" clip-path="url(#terminal-618500324-line-27)">│</text><text class="terminal-618500324-r7" x="366" y="678.8" textLength="1073.6" clip-path="url(#terminal-618500324-line-27)">|&#160;lint-openapi&#160;|&#160;mixed-line-ending&#160;|&#160;pretty-format-json&#160;|&#160;pydocstyle&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="termi [...]
+</text><text class="terminal-618500324-r4" x="0" y="703.2" textLength="12.2" clip-path="url(#terminal-618500324-line-28)">│</text><text class="terminal-618500324-r7" x="366" y="703.2" textLength="1073.6" clip-path="url(#terminal-618500324-line-28)">python-no-log-warn&#160;|&#160;pyupgrade&#160;|&#160;rst-backticks&#160;|&#160;run-flake8&#160;|&#160;run-mypy&#160;|&#160;run-shellcheck&#160;</text><text class="terminal-618500324-r4" x="1451.8" y="703.2" textLength="12.2" clip-path="url(#te [...]
+</text><text class="terminal-618500324-r4" x="0" y="727.6" textLength="12.2" clip-path="url(#terminal-618500324-line-29)">│</text><text class="terminal-618500324-r7" x="366" y="727.6" textLength="1073.6" clip-path="url(#terminal-618500324-line-29)">|&#160;static-check-autoflake&#160;|&#160;trailing-whitespace&#160;|&#160;update-breeze-cmd-output&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-618500324-r4" x="1451.8" y="727 [...]
+</text><text class="terminal-618500324-r4" x="0" y="752" textLength="12.2" clip-path="url(#terminal-618500324-line-30)">│</text><text class="terminal-618500324-r7" x="366" y="752" textLength="1073.6" clip-path="url(#terminal-618500324-line-30)">update-breeze-readme-config-hash&#160;|&#160;update-extras&#160;|&#160;update-in-the-wild-to-be-sorted&#160;|&#160;&#160;&#160;&#160;</text><text class="terminal-618500324-r4" x="1451.8" y="752" textLength="12.2" clip-path="url(#terminal-618500324 [...]
+</text><text class="terminal-618500324-r4" x="0" y="776.4" textLength="12.2" clip-path="url(#terminal-618500324-line-31)">│</text><text class="terminal-618500324-r7" x="366" y="776.4" textLength="1073.6" clip-path="url(#terminal-618500324-line-31)">update-inlined-dockerfile-scripts&#160;|&#160;update-local-yml-file&#160;|&#160;update-migration-references&#160;</text><text class="terminal-618500324-r4" x="1451.8" y="776.4" textLength="12.2" clip-path="url(#terminal-618500324-line-31)">│</ [...]
+</text><text class="terminal-618500324-r4" x="0" y="800.8" textLength="12.2" clip-path="url(#terminal-618500324-line-32)">│</text><text class="terminal-618500324-r7" x="366" y="800.8" textLength="1073.6" clip-path="url(#terminal-618500324-line-32)">|&#160;update-providers-dependencies&#160;|&#160;update-setup-cfg-file&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#16 [...]
+</text><text class="terminal-618500324-r4" x="0" y="825.2" textLength="12.2" clip-path="url(#terminal-618500324-line-33)">│</text><text class="terminal-618500324-r7" x="366" y="825.2" textLength="1073.6" clip-path="url(#terminal-618500324-line-33)">update-spelling-wordlist-to-be-sorted&#160;|&#160;update-supported-versions&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-618500 [...]
+</text><text class="terminal-618500324-r4" x="0" y="849.6" textLength="12.2" clip-path="url(#terminal-618500324-line-34)">│</text><text class="terminal-618500324-r7" x="366" y="849.6" textLength="1073.6" clip-path="url(#terminal-618500324-line-34)">update-vendored-in-k8s-json-schema&#160;|&#160;update-version&#160;|&#160;yamllint&#160;|&#160;yesqa)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-618500324-r [...]
+</text><text class="terminal-618500324-r4" x="0" y="874" textLength="12.2" clip-path="url(#terminal-618500324-line-35)">│</text><text class="terminal-618500324-r5" x="24.4" y="874" textLength="12.2" clip-path="url(#terminal-618500324-line-35)">-</text><text class="terminal-618500324-r5" x="36.6" y="874" textLength="61" clip-path="url(#terminal-618500324-line-35)">-file</text><text class="terminal-618500324-r6" x="317.2" y="874" textLength="24.4" clip-path="url(#terminal-618500324-line-35 [...]
+</text><text class="terminal-618500324-r4" x="0" y="898.4" textLength="12.2" clip-path="url(#terminal-618500324-line-36)">│</text><text class="terminal-618500324-r5" x="24.4" y="898.4" textLength="12.2" clip-path="url(#terminal-618500324-line-36)">-</text><text class="terminal-618500324-r5" x="36.6" y="898.4" textLength="48.8" clip-path="url(#terminal-618500324-line-36)">-all</text><text class="terminal-618500324-r5" x="85.4" y="898.4" textLength="73.2" clip-path="url(#terminal-618500324 [...]
+</text><text class="terminal-618500324-r4" x="0" y="922.8" textLength="12.2" clip-path="url(#terminal-618500324-line-37)">│</text><text class="terminal-618500324-r5" x="24.4" y="922.8" textLength="12.2" clip-path="url(#terminal-618500324-line-37)">-</text><text class="terminal-618500324-r5" x="36.6" y="922.8" textLength="61" clip-path="url(#terminal-618500324-line-37)">-show</text><text class="terminal-618500324-r5" x="97.6" y="922.8" textLength="195.2" clip-path="url(#terminal-618500324 [...]
+</text><text class="terminal-618500324-r4" x="0" y="947.2" textLength="12.2" clip-path="url(#terminal-618500324-line-38)">│</text><text class="terminal-618500324-r5" x="24.4" y="947.2" textLength="12.2" clip-path="url(#terminal-618500324-line-38)">-</text><text class="terminal-618500324-r5" x="36.6" y="947.2" textLength="61" clip-path="url(#terminal-618500324-line-38)">-last</text><text class="terminal-618500324-r5" x="97.6" y="947.2" textLength="85.4" clip-path="url(#terminal-618500324- [...]
+</text><text class="terminal-618500324-r4" x="0" y="971.6" textLength="1464" clip-path="url(#terminal-618500324-line-39)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-618500324-r2" x="1464" y="971.6" textLength="12.2" clip-path="url(#terminal-618500324-line-39)">
+</text><text class="terminal-618500324-r4" x="0" y="996" textLength="24.4" clip-path="url(#terminal-618500324-line-40)">╭─</text><text class="terminal-618500324-r4" x="24.4" y="996" textLength="1415.2" clip-path="url(#terminal-618500324-line-40)">&#160;Options&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-618500324-r4" x="1439.6" y="996" textLength="24.4" clip-path="url(#terminal-618500324-line [...]
+</text><text class="terminal-618500324-r4" x="0" y="1020.4" textLength="12.2" clip-path="url(#terminal-618500324-line-41)">│</text><text class="terminal-618500324-r5" x="24.4" y="1020.4" textLength="12.2" clip-path="url(#terminal-618500324-line-41)">-</text><text class="terminal-618500324-r5" x="36.6" y="1020.4" textLength="85.4" clip-path="url(#terminal-618500324-line-41)">-commit</text><text class="terminal-618500324-r5" x="122" y="1020.4" textLength="48.8" clip-path="url(#terminal-618 [...]
+</text><text class="terminal-618500324-r4" x="0" y="1044.8" textLength="12.2" clip-path="url(#terminal-618500324-line-42)">│</text><text class="terminal-618500324-r2" x="329.4" y="1044.8" textLength="183" clip-path="url(#terminal-618500324-line-42)">exclusive&#160;with&#160;</text><text class="terminal-618500324-r5" x="512.4" y="1044.8" textLength="12.2" clip-path="url(#terminal-618500324-line-42)">-</text><text class="terminal-618500324-r5" x="524.6" y="1044.8" textLength="61" clip-path [...]
+</text><text class="terminal-618500324-r4" x="0" y="1069.2" textLength="12.2" clip-path="url(#terminal-618500324-line-43)">│</text><text class="terminal-618500324-r7" x="329.4" y="1069.2" textLength="1110.2" clip-path="url(#terminal-618500324-line-43)">(TEXT)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
+</text><text class="terminal-618500324-r4" x="0" y="1093.6" textLength="12.2" clip-path="url(#terminal-618500324-line-44)">│</text><text class="terminal-618500324-r5" x="24.4" y="1093.6" textLength="12.2" clip-path="url(#terminal-618500324-line-44)">-</text><text class="terminal-618500324-r5" x="36.6" y="1093.6" textLength="97.6" clip-path="url(#terminal-618500324-line-44)">-verbose</text><text class="terminal-618500324-r6" x="280.6" y="1093.6" textLength="24.4" clip-path="url(#terminal- [...]
+</text><text class="terminal-618500324-r4" x="0" y="1118" textLength="12.2" clip-path="url(#terminal-618500324-line-45)">│</text><text class="terminal-618500324-r5" x="24.4" y="1118" textLength="12.2" clip-path="url(#terminal-618500324-line-45)">-</text><text class="terminal-618500324-r5" x="36.6" y="1118" textLength="48.8" clip-path="url(#terminal-618500324-line-45)">-dry</text><text class="terminal-618500324-r5" x="85.4" y="1118" textLength="48.8" clip-path="url(#terminal-618500324-lin [...]
+</text><text class="terminal-618500324-r4" x="0" y="1142.4" textLength="12.2" clip-path="url(#terminal-618500324-line-46)">│</text><text class="terminal-618500324-r5" x="24.4" y="1142.4" textLength="12.2" clip-path="url(#terminal-618500324-line-46)">-</text><text class="terminal-618500324-r5" x="36.6" y="1142.4" textLength="85.4" clip-path="url(#terminal-618500324-line-46)">-github</text><text class="terminal-618500324-r5" x="122" y="1142.4" textLength="134.2" clip-path="url(#terminal-61 [...]
+</text><text class="terminal-618500324-r4" x="0" y="1166.8" textLength="12.2" clip-path="url(#terminal-618500324-line-47)">│</text><text class="terminal-618500324-r5" x="24.4" y="1166.8" textLength="12.2" clip-path="url(#terminal-618500324-line-47)">-</text><text class="terminal-618500324-r5" x="36.6" y="1166.8" textLength="61" clip-path="url(#terminal-618500324-line-47)">-help</text><text class="terminal-618500324-r6" x="280.6" y="1166.8" textLength="24.4" clip-path="url(#terminal-61850 [...]
+</text><text class="terminal-618500324-r4" x="0" y="1191.2" textLength="1464" clip-path="url(#terminal-618500324-line-48)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-618500324-r2" x="1464" y="1191.2" textLength="12.2" clip-path="url(#terminal-618500324-line-48)">
 </text>
     </g>
     </g>
diff --git a/scripts/ci/docker-compose/remove-sources.yml b/scripts/ci/docker-compose/remove-sources.yml
index 6bdf8a9e1c..65b5bce495 100644
--- a/scripts/ci/docker-compose/remove-sources.yml
+++ b/scripts/ci/docker-compose/remove-sources.yml
@@ -18,9 +18,6 @@
 version: "3.7"
 services:
   airflow:
-    # Forwards local credentials to docker image
-    # Useful for gcloud/aws/kubernetes etc. authorisation to be passed
-    # To inside docker. Use with care - your credentials will be available to
-    # Everything you install in Docker
+    # Removes airflow sources from container
     volumes:
       - ./empty:/opt/airflow/airflow:cached
diff --git a/scripts/ci/pre_commit/common_precommit_utils.py b/scripts/ci/pre_commit/common_precommit_utils.py
index 871aee3672..4b2813b92c 100644
--- a/scripts/ci/pre_commit/common_precommit_utils.py
+++ b/scripts/ci/pre_commit/common_precommit_utils.py
@@ -14,10 +14,11 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-
 from pathlib import Path
 from typing import List
 
+AIRFLOW_SOURCES_ROOT = Path(__file__).parents[3].resolve()
+
 
 def insert_documentation(file_path: Path, content: List[str], header: str, footer: str):
     text = file_path.read_text().splitlines(keepends=True)
@@ -32,4 +33,5 @@ def insert_documentation(file_path: Path, content: List[str], header: str, foote
             replacing = False
         if not replacing:
             result.append(line)
-    file_path.write_text("".join(result))
+    src = "".join(result)
+    file_path.write_text(src)
diff --git a/scripts/ci/pre_commit/pre_commit_build_providers_dependencies.py b/scripts/ci/pre_commit/pre_commit_build_providers_dependencies.py
new file mode 100755
index 0000000000..af2471ad62
--- /dev/null
+++ b/scripts/ci/pre_commit/pre_commit_build_providers_dependencies.py
@@ -0,0 +1,201 @@
+#!/usr/bin/env python
+# 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.
+import json
+import os
+import sys
+from ast import Import, ImportFrom, NodeVisitor, parse
+from collections import defaultdict
+from pathlib import Path
+from typing import Any, Dict, List, Optional
+
+import yaml
+from rich.console import Console
+
+console = Console(color_system="standard", width=200)
+
+AIRFLOW_PROVIDERS_IMPORT_PREFIX = "airflow.providers."
+
+AIRFLOW_SOURCES_ROOT = Path(__file__).parents[3].resolve()
+
+AIRFLOW_PROVIDERS_DIR = AIRFLOW_SOURCES_ROOT / "airflow" / "providers"
+AIRFLOW_TESTS_PROVIDERS_DIR = AIRFLOW_SOURCES_ROOT / "tests" / "providers"
+AIRFLOW_SYSTEM_TESTS_PROVIDERS_DIR = AIRFLOW_SOURCES_ROOT / "system" / "tests" / "providers"
+
+DEPENDENCIES_JSON_FILE_PATH = AIRFLOW_SOURCES_ROOT / "generated" / "provider_dependencies.json"
+
+sys.path.insert(0, str(AIRFLOW_SOURCES_ROOT))  # make sure setup is imported from Airflow
+
+warnings: List[str] = []
+errors: List[str] = []
+
+CROSS_PROVIDERS_DEPS = "cross-providers-deps"
+DEPS = "deps"
+
+ALL_DEPENDENCIES: Dict[str, Dict[str, List[str]]] = defaultdict(lambda: defaultdict(list))
+
+ALL_PROVIDERS: Dict[str, Dict[str, Any]] = defaultdict(lambda: defaultdict())
+ALL_PROVIDER_FILES: List[Path] = []
+
+# Allow AST to parse the files.
+sys.path.append(str(AIRFLOW_SOURCES_ROOT))
+
+
+class ImportFinder(NodeVisitor):
+    """
+    AST visitor that collects all imported names in its imports
+    """
+
+    def __init__(self) -> None:
+        self.imports: List[str] = []
+        self.handled_import_exception = List[str]
+        self.tried_imports: List[str] = []
+
+    def process_import(self, import_name: str) -> None:
+        self.imports.append(import_name)
+
+    def get_import_name_from_import_from(self, node: ImportFrom) -> List[str]:
+        import_names: List[str] = []
+        for alias in node.names:
+            name = alias.name
+            fullname = f'{node.module}.{name}' if node.module else name
+            import_names.append(fullname)
+        return import_names
+
+    def visit_Import(self, node: Import):
+        for alias in node.names:
+            self.process_import(alias.name)
+
+    def visit_ImportFrom(self, node: ImportFrom):
+        if node.module == '__future__':
+            return
+        for fullname in self.get_import_name_from_import_from(node):
+            self.process_import(fullname)
+
+
+def find_all_providers_and_provider_files():
+    for (root, _, filenames) in os.walk(AIRFLOW_PROVIDERS_DIR):
+        for filename in filenames:
+            if filename == 'provider.yaml':
+                provider_file = Path(root, filename)
+                provider_name = str(provider_file.parent.relative_to(AIRFLOW_PROVIDERS_DIR)).replace(
+                    os.sep, "."
+                )
+                ALL_PROVIDERS[provider_name] = yaml.safe_load(provider_file.read_text())
+            path = Path(root, filename)
+            if path.is_file() and path.name.endswith(".py"):
+                ALL_PROVIDER_FILES.append(Path(root, filename))
+
+
+def get_provider_id_from_relative_import_or_file(relative_path_or_file: str) -> Optional[str]:
+    provider_candidate = relative_path_or_file.replace(os.sep, ".").split(".")
+    while len(provider_candidate) > 0:
+        candidate_provider_id = ".".join(provider_candidate)
+        if candidate_provider_id in ALL_PROVIDERS:
+            return candidate_provider_id
+        provider_candidate = provider_candidate[:-1]
+    return None
+
+
+def get_provider_id_from_import(import_name: str, file_path: Path) -> Optional[str]:
+    if not import_name.startswith(AIRFLOW_PROVIDERS_IMPORT_PREFIX):
+        # skip silently - it's OK to get non-provider imports
+        return None
+    relative_provider_import = import_name[len(AIRFLOW_PROVIDERS_IMPORT_PREFIX) :]
+    provider_id = get_provider_id_from_relative_import_or_file(relative_provider_import)
+    if provider_id is None:
+        warnings.append(f"We could not determine provider id from import {import_name} in {file_path}")
+    return provider_id
+
+
+def get_imports_from_file(file_path: Path) -> List[str]:
+    root = parse(file_path.read_text(), file_path.name)
+    visitor = ImportFinder()
+    visitor.visit(root)
+    return visitor.imports
+
+
+def get_provider_id_from_file_name(file_path: Path) -> Optional[str]:
+    # is_relative_to is only available in Python 3.9 - we should simplify this check when we are Python 3.9+
+    try:
+        relative_path = file_path.relative_to(AIRFLOW_PROVIDERS_DIR)
+    except ValueError:
+        try:
+            relative_path = file_path.relative_to(AIRFLOW_SYSTEM_TESTS_PROVIDERS_DIR)
+        except ValueError:
+            try:
+                relative_path = file_path.relative_to(AIRFLOW_TESTS_PROVIDERS_DIR)
+            except ValueError:
+                errors.append(f"Wrong file not in the providers package = {file_path}")
+                return None
+    provider_id = get_provider_id_from_relative_import_or_file(str(relative_path))
+    if provider_id is None and file_path.name not in ["__init__.py", "get_provider_info.py"]:
+        warnings.append(f"We had a problem to classify the file {file_path} to a provider")
+    return provider_id
+
+
+def check_if_different_provider_used(file_path: Path) -> None:
+    file_provider = get_provider_id_from_file_name(file_path)
+    if not file_provider:
+        return
+    imports = get_imports_from_file(file_path)
+    for import_name in imports:
+        imported_provider = get_provider_id_from_import(import_name, file_path)
+        if imported_provider is not None and imported_provider not in ALL_PROVIDERS:
+            warnings.append(f"The provider {imported_provider} from {file_path} cannot be found.")
+        elif imported_provider and file_provider != imported_provider:
+            ALL_DEPENDENCIES[file_provider][CROSS_PROVIDERS_DEPS].append(imported_provider)
+
+
+if __name__ == '__main__':
+    find_all_providers_and_provider_files()
+    num_files = len(ALL_PROVIDER_FILES)
+    num_providers = len(ALL_PROVIDERS)
+    console.print(f"Found {len(ALL_PROVIDERS)} providers with {len(ALL_PROVIDER_FILES)} Python files.")
+
+    for file in ALL_PROVIDER_FILES:
+        check_if_different_provider_used(file)
+
+    for provider, provider_yaml_content in ALL_PROVIDERS.items():
+        ALL_DEPENDENCIES[provider][DEPS].extend(provider_yaml_content['dependencies'])
+
+    if warnings:
+        console.print("[yellow]Warnings!\n")
+        for warning in warnings:
+            console.print(f"[yellow] {warning}")
+        console.print(f"[bright_blue]Total: {len(warnings)} warnings.")
+    if errors:
+        console.print("[red]Errors!\n")
+        for error in errors:
+            console.print(f"[red] {error}")
+        console.print(f"[bright_blue]Total: {len(errors)} errors.")
+    unique_sorted_dependencies: Dict[str, Dict[str, List[str]]] = defaultdict(dict)
+    for key in sorted(ALL_DEPENDENCIES.keys()):
+        unique_sorted_dependencies[key][DEPS] = sorted(ALL_DEPENDENCIES[key][DEPS])
+        unique_sorted_dependencies[key][CROSS_PROVIDERS_DEPS] = sorted(
+            set(ALL_DEPENDENCIES[key][CROSS_PROVIDERS_DEPS])
+        )
+    if errors:
+        console.print()
+        console.print("[red]Errors found during verification. Exiting!")
+        console.print()
+        sys.exit(1)
+    DEPENDENCIES_JSON_FILE_PATH.write_text(json.dumps(unique_sorted_dependencies, indent=2) + "\n")
+    console.print()
+    console.print("[green]Verification complete! Success!\n")
+    console.print(f"Written {DEPENDENCIES_JSON_FILE_PATH}")
+    console.print()
diff --git a/scripts/ci/pre_commit/pre_commit_build_providers_dependencies.sh b/scripts/ci/pre_commit/pre_commit_build_providers_dependencies.sh
deleted file mode 100755
index 3aa606b6c5..0000000000
--- a/scripts/ci/pre_commit/pre_commit_build_providers_dependencies.sh
+++ /dev/null
@@ -1,36 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-set -euo pipefail
-
-PRE_COMMIT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
-readonly PRE_COMMIT_DIR
-
-AIRFLOW_SOURCES=$(cd "${PRE_COMMIT_DIR}/../../../" && pwd);
-readonly AIRFLOW_SOURCES
-cd "${AIRFLOW_SOURCES}" || exit 1
-
-export PRINT_INFO_FROM_SCRIPTS="false"
-export SKIP_CHECK_REMOTE_IMAGE="true"
-
-PYTHONPATH="$(pwd)"
-export PYTHONPATH
-
-find airflow/providers -name '*.py' -print0 | \
-    xargs -0 python3 tests/build_provider_packages_dependencies.py \
-        --provider-dependencies-file "airflow/providers/dependencies.json" \
-        --documentation-file CONTRIBUTING.rst
diff --git a/scripts/ci/pre_commit/pre_commit_check_extras_have_providers.py b/scripts/ci/pre_commit/pre_commit_check_extras_have_providers.py
deleted file mode 100755
index 91943d52e8..0000000000
--- a/scripts/ci/pre_commit/pre_commit_check_extras_have_providers.py
+++ /dev/null
@@ -1,86 +0,0 @@
-#!/usr/bin/env python
-# 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.
-
-# Check if the extras have providers defined.
-import os
-import sys
-from os.path import dirname
-from pathlib import Path
-from typing import List
-
-from rich import print
-
-AIRFLOW_SOURCES_DIR = os.path.abspath(os.path.join(dirname(__file__), os.pardir, os.pardir, os.pardir))
-
-sys.path.insert(0, AIRFLOW_SOURCES_DIR)
-# flake8: noqa: F401
-
-from setup import ALL_PROVIDERS  # noqa
-
-sys.path.append(AIRFLOW_SOURCES_DIR)
-
-errors: List[str] = []
-
-PROVIDERS_DIR = os.path.join(AIRFLOW_SOURCES_DIR, "airflow", "providers")
-
-
-def get_provider_directory(provider: str) -> str:
-    """Returns provider directory derived from name"""
-    return os.path.join(PROVIDERS_DIR, *provider.split('.'))
-
-
-def check_all_providers_listed_have_directory() -> None:
-    for provider in ALL_PROVIDERS:
-        provider_directory = get_provider_directory(provider)
-        if not os.path.isdir(provider_directory):
-            errors.append(
-                f"The provider {provider} is defined in setup.py: [bold]PROVIDERS_REQUIREMENTS[/] but it "
-                + f"has missing {provider_directory} directory: [red]NOK[/]"
-            )
-            continue
-        if not os.path.exists(os.path.join(provider_directory, "__init__.py")):
-            errors.append(
-                f"The {provider} does not have the __init__.py "
-                + f"file in the {provider_directory} directory [red]NOK[/]"
-            )
-        if not os.path.exists(os.path.join(provider_directory, "provider.yaml")):
-            errors.append(
-                f"The provider {provider} does not have the provider.yaml "
-                + f"in the {provider_directory} directory: [red]NOK[/]"
-            )
-
-
-def check_all_providers_are_listed_in_setup_py() -> None:
-    for path in Path(PROVIDERS_DIR).rglob('provider.yaml'):
-        provider_name = str(path.parent.relative_to(PROVIDERS_DIR)).replace(os.sep, ".")
-        if provider_name not in ALL_PROVIDERS:
-            errors.append(
-                f"The provider {provider_name} is missing in setup.py "
-                "[bold]PROVIDERS_REQUIREMENTS[/]: [red]NOK[/]"
-            )
-
-
-if __name__ == '__main__':
-    check_all_providers_listed_have_directory()
-    check_all_providers_are_listed_in_setup_py()
-    if errors:
-        for message in errors:
-            print(message, file=sys.stderr)
-        sys.exit(1)
-    else:
-        print("All providers are correctly defined in setup.py [green]OK[/]")
diff --git a/scripts/ci/pre_commit/pre_commit_check_order_setup.py b/scripts/ci/pre_commit/pre_commit_check_order_setup.py
index 8e4ac563f6..b0b5f1cd95 100755
--- a/scripts/ci/pre_commit/pre_commit_check_order_setup.py
+++ b/scripts/ci/pre_commit/pre_commit_check_order_setup.py
@@ -142,9 +142,8 @@ if __name__ == '__main__':
         file_contents = setup_file.read()
     check_main_dependent_group(file_contents)
     check_alias_dependent_group(file_contents)
-    check_variable_order("PROVIDERS_REQUIREMENTS")
-    check_variable_order("CORE_EXTRAS_REQUIREMENTS")
-    check_variable_order("ADDITIONAL_EXTRAS_REQUIREMENTS")
+    check_variable_order("CORE_EXTRAS_DEPENDENCIES")
+    check_variable_order("ADDITIONAL_EXTRAS_DEPENDENCIES")
     check_variable_order("EXTRAS_DEPRECATED_ALIASES")
     check_variable_order("PREINSTALLED_PROVIDERS")
     check_install_and_setup_requires()
diff --git a/scripts/ci/pre_commit/pre_commit_check_setup_extra_packages_ref.py b/scripts/ci/pre_commit/pre_commit_check_setup_extra_packages_ref.py
index 28d2bbc7c9..5417b624fd 100755
--- a/scripts/ci/pre_commit/pre_commit_check_setup_extra_packages_ref.py
+++ b/scripts/ci/pre_commit/pre_commit_check_setup_extra_packages_ref.py
@@ -40,8 +40,7 @@ sys.path.insert(0, AIRFLOW_SOURCES_DIR)
 from setup import (  # noqa # isort:skip
     add_all_provider_packages,
     EXTRAS_DEPRECATED_ALIASES,
-    EXTRAS_REQUIREMENTS,
-    PROVIDERS_REQUIREMENTS,
+    EXTRAS_DEPENDENCIES,
     PREINSTALLED_PROVIDERS,
 )
 
@@ -54,7 +53,7 @@ def get_file_content(*path_elements: str) -> str:
 
 def get_extras_from_setup() -> Set[str]:
     """Returns a set of regular (non-deprecated) extras from setup."""
-    return set(EXTRAS_REQUIREMENTS.keys()) - set(EXTRAS_DEPRECATED_ALIASES.keys())
+    return set(EXTRAS_DEPENDENCIES.keys()) - set(EXTRAS_DEPRECATED_ALIASES.keys())
 
 
 def get_extras_from_docs() -> Set[str]:
@@ -125,8 +124,7 @@ def check_extras(console: Console) -> bool:
             f"""\
 [red bold]ERROR!![/red bold]
 
-The "[bold]CORE_EXTRAS_REQUIREMENTS[/bold]", "[bold]ADDITIONAL_PROVIDERS_REQUIREMENTS[/bold]", and
-    "[bold]PROVIDERS_REQUIREMENTS[/bold]"
+The "[bold]CORE_EXTRAS_DEPENDENCIES[/bold]"
 sections in the setup file: [bold yellow]{SETUP_PY_FILE}[/bold yellow]
 should be synchronized with the "Extra Packages Reference"
 in the documentation file: [bold yellow]{DOCS_FILE}[/bold yellow].
diff --git a/scripts/ci/pre_commit/pre_commit_insert_extras.py b/scripts/ci/pre_commit/pre_commit_insert_extras.py
index dbcc403900..c0c19f4ab0 100755
--- a/scripts/ci/pre_commit/pre_commit_insert_extras.py
+++ b/scripts/ci/pre_commit/pre_commit_insert_extras.py
@@ -26,7 +26,7 @@ sys.path.insert(0, str(AIRFLOW_SOURCES_DIR))  # make sure setup is imported from
 # flake8: noqa: F401
 
 from common_precommit_utils import insert_documentation  # isort: skip
-from setup import EXTRAS_REQUIREMENTS  # isort:skip
+from setup import EXTRAS_DEPENDENCIES  # isort:skip
 
 sys.path.append(str(AIRFLOW_SOURCES_DIR))
 
@@ -52,8 +52,8 @@ if __name__ == '__main__':
     global_constants_file_path = (
         AIRFLOW_SOURCES_DIR / "dev" / "breeze" / "src" / "airflow_breeze" / "global_constants.py"
     )
-    extras_list = wrap(", ".join(EXTRAS_REQUIREMENTS.keys()), 100)
+    extras_list = wrap(", ".join(EXTRAS_DEPENDENCIES.keys()), 100)
     extras_list = [line + "\n" for line in extras_list]
-    extras_code = [f"    {extra}\n" for extra in EXTRAS_REQUIREMENTS.keys()]
+    extras_code = [f"    {extra}\n" for extra in EXTRAS_DEPENDENCIES.keys()]
     insert_documentation(install_file_path, extras_list, INSTALL_HEADER, INSTALL_FOOTER)
     insert_documentation(contributing_file_path, extras_list, RST_HEADER, RST_FOOTER)
diff --git a/scripts/in_container/_in_container_utils.sh b/scripts/in_container/_in_container_utils.sh
index 66f2e6b083..33a7b67d28 100644
--- a/scripts/in_container/_in_container_utils.sh
+++ b/scripts/in_container/_in_container_utils.sh
@@ -314,7 +314,7 @@ function install_local_airflow_with_eager_upgrade() {
 
 
 function install_all_providers_from_pypi_with_eager_upgrade() {
-    NO_PROVIDERS_EXTRAS=$(python -c 'import setup; print(",".join(setup.CORE_EXTRAS_REQUIREMENTS))')
+    NO_PROVIDERS_EXTRAS=$(python -c 'import setup; print(",".join(setup.CORE_EXTRAS_DEPENDENCIES))')
     ALL_PROVIDERS_PACKAGES=$(python -c 'import setup; print(setup.get_all_provider_packages())')
     local packages_to_install=()
     local provider_package
diff --git a/scripts/in_container/run_generate_constraints.sh b/scripts/in_container/run_generate_constraints.sh
index e85c2fb9c9..d91cbf81b6 100755
--- a/scripts/in_container/run_generate_constraints.sh
+++ b/scripts/in_container/run_generate_constraints.sh
@@ -25,7 +25,7 @@ mkdir -pv "${CONSTRAINTS_DIR}"
 
 
 if [[ ${AIRFLOW_CONSTRAINTS_MODE} == "constraints-no-providers" ]]; then
-    NO_PROVIDERS_EXTRAS=$(python -c 'import setup; print(",".join(setup.CORE_EXTRAS_REQUIREMENTS.keys()))')
+    NO_PROVIDERS_EXTRAS=$(python -c 'import setup; print(",".join(setup.CORE_EXTRAS_DEPENDENCIES.keys()))')
     CURRENT_CONSTRAINT_FILE="${CONSTRAINTS_DIR}/${AIRFLOW_CONSTRAINTS_MODE}-${PYTHON_MAJOR_MINOR_VERSION}.txt"
     echo
     echo "UnInstall All PIP packages."
diff --git a/scripts/in_container/verify_providers.py b/scripts/in_container/verify_providers.py
index 9c5595b6f5..fa4f759dec 100755
--- a/scripts/in_container/verify_providers.py
+++ b/scripts/in_container/verify_providers.py
@@ -283,9 +283,9 @@ def get_all_providers() -> List[str]:
     Returns all providers for regular packages.
     :return: list of providers that are considered for provider packages
     """
-    from setup import PROVIDERS_REQUIREMENTS
+    from setup import ALL_PROVIDERS
 
-    return list(PROVIDERS_REQUIREMENTS.keys())
+    return list(ALL_PROVIDERS)
 
 
 def import_all_classes(
@@ -380,16 +380,6 @@ def import_all_classes(
         return imported_classes, all_warnings
 
 
-def get_provider_packages() -> List[str]:
-    """
-    Returns all provider packages.
-
-    """
-    from setup import PROVIDERS_REQUIREMENTS
-
-    return list(PROVIDERS_REQUIREMENTS.keys())
-
-
 def is_imported_from_same_module(the_class: str, imported_name: str) -> bool:
     """
     Is the class imported from another module?
diff --git a/setup.cfg b/setup.cfg
index 41f0a05d1e..754cb00271 100644
--- a/setup.cfg
+++ b/setup.cfg
@@ -172,6 +172,10 @@ airflow.serialization=*.json
 airflow.utils=
     context.pyi
 
+[options.data_files]
+generated=
+    generated/provider_dependencies.json
+
 [options.entry_points]
 console_scripts=
     airflow=airflow.__main__:main
diff --git a/setup.py b/setup.py
index 996c66c736..991b6b0617 100644
--- a/setup.py
+++ b/setup.py
@@ -17,15 +17,17 @@
 # under the License.
 """Setup.py for the Airflow project."""
 import glob
+import json
 import logging
 import os
 import subprocess
 import sys
 import unittest
 from copy import deepcopy
-from os.path import dirname, relpath
+from os.path import relpath
+from pathlib import Path
 from textwrap import wrap
-from typing import Dict, List
+from typing import Dict, Iterable, List, Set
 
 from setuptools import Command, Distribution, find_namespace_packages, setup
 from setuptools.command.develop import develop as develop_orig
@@ -35,6 +37,7 @@ from setuptools.command.install import install as install_orig
 # stdlib, which is deprecated in Python 3.10 and will be removed in 3.12.
 from distutils import log  # isort: skip
 
+
 # Controls whether providers are installed from packages or directly from sources
 # It is turned on by default in case of development environments such as Breeze
 # And it is particularly useful when you add a new provider and there is no
@@ -46,13 +49,36 @@ logger = logging.getLogger(__name__)
 
 version = '2.4.0.dev0'
 
-my_dir = dirname(__file__)
+AIRFLOW_SOURCES_ROOT = Path(__file__).parent.resolve()
+PROVIDERS_ROOT = AIRFLOW_SOURCES_ROOT / "airflow" / "providers"
+
+CROSS_PROVIDERS_DEPS = "cross-providers-deps"
+DEPS = "deps"
+
+
+#
+# NOTE! IN Airflow 2.4.+ dependencies for providers are maintained in `provider.yaml` files for each
+# provider separately. They are loaded here and if you want to modify them, you need to modify
+# corresponding provider.yaml file.
+#
+def fill_provider_dependencies() -> Dict[str, Dict[str, List[str]]]:
+    try:
+        return json.loads((AIRFLOW_SOURCES_ROOT / "generated" / "provider_dependencies.json").read_text())
+    except Exception as e:
+        print(f"Exception while loading provider dependencies {e}")
+        # we can ignore loading dependencies when they are missing - they are only used to generate
+        # correct extras when packages are build and when we install airflow from sources
+        # (in both cases the provider_dependencies should be present).
+        return {}
+
+
+PROVIDER_DEPENDENCIES = fill_provider_dependencies()
 
 
 def airflow_test_suite() -> unittest.TestSuite:
     """Test suite for Airflow tests"""
     test_loader = unittest.TestLoader()
-    test_suite = test_loader.discover(os.path.join(my_dir, 'tests'), pattern='test_*.py')
+    test_suite = test_loader.discover(str(AIRFLOW_SOURCES_ROOT / 'tests'), pattern='test_*.py')
     return test_suite
 
 
@@ -82,7 +108,7 @@ class CleanCommand(Command):
 
     def run(self) -> None:
         """Remove temporary files and directories."""
-        os.chdir(my_dir)
+        os.chdir(str(AIRFLOW_SOURCES_ROOT))
         self.rm_all_files(glob.glob('./build/*'))
         self.rm_all_files(glob.glob('./**/__pycache__/*', recursive=True))
         self.rm_all_files(glob.glob('./**/*.pyc', recursive=True))
@@ -129,7 +155,7 @@ class ListExtras(Command):
 
     def run(self) -> None:
         """List extras."""
-        print("\n".join(wrap(", ".join(EXTRAS_REQUIREMENTS.keys()), 100)))
+        print("\n".join(wrap(", ".join(EXTRAS_DEPENDENCIES.keys()), 100)))
 
 
 def git_version(version_: str) -> str:
@@ -149,8 +175,8 @@ def git_version(version_: str) -> str:
         import git
 
         try:
-            repo = git.Repo(os.path.join(*[my_dir, '.git']))
-        except git.NoSuchPathError:
+            repo = git.Repo(str(AIRFLOW_SOURCES_ROOT / '.git'))
+        except (git.NoSuchPathError):
             logger.warning('.git directory not found: Cannot compute the git version')
             return ''
         except git.InvalidGitRepositoryError:
@@ -168,7 +194,7 @@ def git_version(version_: str) -> str:
     return 'no_git_version'
 
 
-def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version"])) -> None:
+def write_version(filename: str = str(AIRFLOW_SOURCES_ROOT / "airflow" / "git_version")) -> None:
     """
     Write the Semver version + git hash to file, e.g. ".dev0+2f635dc265e78db6708f59f68e8009abb92c1e65".
 
@@ -179,34 +205,13 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version
         file.write(text)
 
 
-pandas_requirement = 'pandas>=0.17.1'
-
+#
+# NOTE! IN Airflow 2.4.+ dependencies for providers are maintained in `provider.yaml` files for each
+# provider separately. Before, the provider dependencies were kept here. THEY ARE NOT HERE ANYMORE.
+#
 # 'Start dependencies group' and 'Start dependencies group' are mark for ./scripts/ci/check_order_setup.py
 # If you change this mark you should also change ./scripts/ci/check_order_setup.py
 # Start dependencies group
-alibaba = [
-    'oss2>=2.14.0',
-]
-amazon = [
-    'boto3>=1.15.0',
-    # watchtower 3 has been released end Jan and introduced breaking change across the board that might
-    # change logging behaviour:
-    # https://github.com/kislyuk/watchtower/blob/develop/Changes.rst#changes-for-v300-2022-01-26
-    # TODO: update to watchtower >3
-    'watchtower~=2.0.1',
-    'jsonpath_ng>=1.5.3',
-    'redshift_connector>=2.0.888',
-    'sqlalchemy_redshift>=0.8.6',
-    pandas_requirement,
-    'mypy-boto3-rds>=1.21.0',
-    'mypy-boto3-redshift-data>=1.21.0',
-    'mypy-boto3-appflow>=1.21.0',
-]
-apache_beam = [
-    'apache-beam>=2.39.0',
-]
-arangodb = ['python-arango>=7.3.2']
-asana = ['asana>=0.10']
 async_packages = [
     'eventlet>=0.9.7',
     'gevent>=0.13',
@@ -215,29 +220,6 @@ async_packages = [
 atlas = [
     'atlasclient>=0.1.2',
 ]
-azure = [
-    'azure-batch>=8.0.0',
-    'azure-cosmos>=4.0.0',
-    'azure-datalake-store>=0.0.45',
-    'azure-identity>=1.3.1',
-    'azure-keyvault-secrets>=4.1.0,<5.0',
-    'azure-kusto-data>=0.0.43,<0.1',
-    # Azure integration uses old librarires and the limits below reflect that
-    # TODO: upgrade to newer versions of all the below libraries
-    'azure-mgmt-containerinstance>=1.5.0,<2.0',
-    'azure-mgmt-datafactory>=1.0.0,<2.0',
-    'azure-mgmt-datalake-store>=0.5.0',
-    'azure-mgmt-resource>=2.2.0',
-    # limited due to https://github.com/Azure/azure-sdk-for-python/pull/18801  implementation released in 12.9
-    'azure-storage-blob>=12.7.0,<12.9.0',
-    'azure-storage-common>=2.1.0',
-    'azure-storage-file>=2.1.0',
-    # Limited due to https://github.com/Azure/azure-uamqp-python/issues/191
-    'azure-servicebus>=7.6.1; platform_machine != "aarch64"',
-]
-cassandra = [
-    'cassandra-driver>=3.13.0',
-]
 celery = [
     # The Celery is known to introduce problems when upgraded to a MAJOR version. Airflow Core
     # Uses Celery for CeleryExecutor, and we also know that Kubernetes Python client follows SemVer
@@ -245,19 +227,15 @@ celery = [
     # This is a crucial component of Airflow, so we should limit it to the next MAJOR version and only
     # deliberately bump the version when we tested it, and we know it can be bumped.
     # Bumping this version should also be connected with
-    # limiting minimum airflow version supported in cncf.kubernetes provider, due to the
+    # limiting minimum airflow version supported in celery provider due to the
     # potential breaking changes in Airflow Core as well (celery is added as extra, so Airflow
-    # core is not hard-limited via install-requirements, only by extra).
-    'celery>=5.2.3,<6',
-    'flower>=1.0.0',
+    # core is not hard-limited via install-requires, only by extra).
+    'celery>=5.2.3,<6'
 ]
 cgroups = [
     # Cgroupspy 0.2.2 added Python 3.10 compatibility
     'cgroupspy>=0.2.2',
 ]
-cloudant = [
-    'cloudant>=2.0',
-]
 dask = [
     # Dask support is limited, we need Dask team to upgrade support for dask if we were to continue
     # Supporting it in the future
@@ -265,20 +243,11 @@ dask = [
     'dask>=2.9.0',
     'distributed>=2.11.1',
 ]
-databricks = [
-    'requests>=2.27,<3',
-    'databricks-sql-connector>=2.0.0, <3.0.0',
-    'aiohttp>=3.6.3, <4',
-]
-datadog = [
-    'datadog>=0.14.0',
-]
 deprecated_api = [
     'requests>=2.26.0',
 ]
 doc = [
     'click>=8.0',
-    'sphinx>=4.4.0',
     # Docutils 0.17.0 converts generated <div class="section"> into <section> and breaks our doc formatting
     # By adding a lot of whitespace separation. This limit can be lifted when we update our doc to handle
     # <section> tags for sections
@@ -292,143 +261,20 @@ doc = [
     'sphinx-copybutton',
     'sphinx-jinja>=2.0',
     'sphinx-rtd-theme>=0.1.6',
+    'sphinx>=4.4.0',
     'sphinxcontrib-httpdomain>=1.7.0',
     'sphinxcontrib-redoc>=1.6.0',
     'sphinxcontrib-spelling>=7.3',
 ]
-docker = [
-    'docker>=5.0.3',
-]
-drill = ['sqlalchemy-drill>=1.1.0', 'sqlparse>=0.4.1']
-druid = [
-    'pydruid>=0.4.1',
-]
-elasticsearch = [
-    'elasticsearch>7',
-    'elasticsearch-dbapi',
-    'elasticsearch-dsl>=5.0.0',
-]
-exasol = ['pyexasol>=0.5.1', pandas_requirement]
-facebook = [
-    'facebook-business>=6.0.2',
-]
 flask_appbuilder_oauth = [
     'flask-appbuilder[oauth]',
 ]
-github = [
-    'pygithub',
-]
-google = [
-    # Google has very clear rules on what dependencies should be used. All the limits below
-    # follow strict guidelines of Google Libraries as quoted here:
-    # While this issue is open, dependents of google-api-core, google-cloud-core. and google-auth
-    # should preserve >1, <3 pins on these packages.
-    # https://github.com/googleapis/google-cloud-python/issues/10566
-    # Some of Google Packages are limited to <2.0.0 because 2.0.0 releases of the libraries
-    # Introduced breaking changes across the board. Those libraries should be upgraded soon
-    # TODO: Upgrade all Google libraries that are limited to <2.0.0
-    'PyOpenSSL',
-    'google-ads>=15.1.1',
-    'google-api-core>=2.7.0,<3.0.0',
-    'google-api-python-client>=1.6.0,<2.0.0',
-    'google-auth>=1.0.0',
-    'google-auth-httplib2>=0.0.1',
-    'google-cloud-aiplatform>=1.7.1,<2.0.0',
-    'google-cloud-automl>=2.1.0',
-    'google-cloud-bigquery-datatransfer>=3.0.0',
-    'google-cloud-bigtable>=1.0.0,<2.0.0',
-    'google-cloud-build>=3.0.0',
-    'google-cloud-container>=2.2.0,<3.0.0',
-    'google-cloud-datacatalog>=3.0.0',
-    'google-cloud-dataplex>=0.1.0',
-    'google-cloud-dataproc>=3.1.0',
-    'google-cloud-dataproc-metastore>=1.2.0,<2.0.0',
-    'google-cloud-dlp>=0.11.0,<2.0.0',
-    'google-cloud-kms>=2.0.0',
-    'google-cloud-language>=1.1.1,<2.0.0',
-    'google-cloud-logging>=2.1.1',
-    'google-cloud-memcache>=0.2.0',
-    'google-cloud-monitoring>=2.0.0',
-    'google-cloud-os-login>=2.0.0',
-    'google-cloud-orchestration-airflow>=1.0.0,<2.0.0',
-    'google-cloud-pubsub>=2.0.0',
-    'google-cloud-redis>=2.0.0',
-    'google-cloud-secret-manager>=0.2.0,<2.0.0',
-    'google-cloud-spanner>=1.10.0,<2.0.0',
-    'google-cloud-speech>=0.36.3,<2.0.0',
-    'google-cloud-storage>=1.30,<2.0.0',
-    'google-cloud-tasks>=2.0.0',
-    'google-cloud-texttospeech>=0.4.0,<2.0.0',
-    'google-cloud-translate>=1.5.0,<2.0.0',
-    'google-cloud-videointelligence>=1.7.0,<2.0.0',
-    'google-cloud-vision>=0.35.2,<2.0.0',
-    'google-cloud-workflows>=0.1.0,<2.0.0',
-    'grpcio-gcp>=0.2.2',
-    'httpx',
-    'json-merge-patch>=0.2',
-    'looker-sdk>=22.2.0',
-    'pandas-gbq',
-    pandas_requirement,
-    'sqlalchemy-bigquery>=1.2.1',
-    # A transient dependency of google-cloud-bigquery-datatransfer, but we
-    # further constrain it since older versions are buggy.
-    'proto-plus>=1.19.6',
-]
-grpc = [
-    # Google has very clear rules on what dependencies should be used. All the limits below
-    # follow strict guidelines of Google Libraries as quoted here:
-    # While this issue is open, dependents of google-api-core, google-cloud-core. and google-auth
-    # should preserve >1, <3 pins on these packages.
-    # https://github.com/googleapis/google-cloud-python/issues/10566
-    'google-auth>=1.0.0, <3.0.0',
-    'google-auth-httplib2>=0.0.1',
-    'grpcio>=1.15.0',
-]
-hashicorp = [
-    'hvac>=0.10',
-]
-hdfs = [
-    'snakebite-py3',
-    'hdfs[avro,dataframe,kerberos]>=2.0.4',
-]
-hive = [
-    'hmsclient>=0.1.0',
-    'pyhive[hive]>=0.6.0',
-    # in case of Python 3.9 sasl library needs to be installed with version higher or equal than
-    # 0.3.1 because only that version supports Python 3.9. For other Python version pyhive[hive] pulls
-    # the sasl library anyway (and there sasl library version is not relevant)
-    'sasl>=0.3.1; python_version>="3.9"',
-    'thrift>=0.9.2',
-    pandas_requirement,
-]
-http = [
-    # The 2.26.0 release of requests got rid of the chardet LGPL mandatory dependency, allowing us to
-    # release it as a requirement for airflow
-    'requests>=2.26.0',
-]
-http_provider = [
-    'apache-airflow-providers-http',
-]
-influxdb = [
-    'influxdb-client>=1.19.0',
-    pandas_requirement,
-]
-jdbc = [
-    'jaydebeapi>=1.1.1',
-]
-jenkins = [
-    'python-jenkins>=1.0.0',
-]
-jira = [
-    'JIRA>1.0.7',
-]
 kerberos = [
     'pykerberos>=1.1.13',
     'requests_kerberos>=0.10.0',
     'thrift_sasl>=0.2.0',
 ]
 kubernetes = [
-    'cryptography>=2.0.0',
     # The Kubernetes API is known to introduce problems when upgraded to a MAJOR version. Airflow Core
     # Uses Kubernetes for Kubernetes executor, and we also know that Kubernetes Python client follows SemVer
     # (https://github.com/kubernetes-client/python#compatibility). This is a crucial component of Airflow
@@ -436,143 +282,38 @@ kubernetes = [
     # tested it, and we know it can be bumped. Bumping this version should also be connected with
     # limiting minimum airflow version supported in cncf.kubernetes provider, due to the
     # potential breaking changes in Airflow Core as well (kubernetes is added as extra, so Airflow
-    # core is not hard-limited via install-requirements, only by extra).
+    # core is not hard-limited via install-requires, only by extra).
+    'cryptography>=2.0.0',
     'kubernetes>=21.7.0,<24',
 ]
-kylin = ['kylinpy>=2.6']
 ldap = [
     'ldap3>=2.5.1',
     'python-ldap',
 ]
 leveldb = ['plyvel; platform_machine != "aarch64"']
-mongo = [
-    'dnspython>=1.13.0',
-    # pymongo 4.0.0 removes connection option `ssl_cert_reqs` which is used in providers-mongo/2.2.0
-    # TODO: Upgrade to pymongo 4.0.0+
-    'pymongo>=3.6.0,<4.0.0',
-]
-mssql = [
-    'pymssql>=2.1.5; platform_machine != "aarch64"',
-]
-mysql = [
-    'mysql-connector-python>=8.0.11; platform_machine != "aarch64"',
-    'mysqlclient>=1.3.6; platform_machine != "aarch64"',
-]
-neo4j = ['neo4j>=4.2.1']
-odbc = [
-    'pyodbc',
-]
-opsgenie = [
-    'opsgenie-sdk>=2.1.5',
-]
-oracle = [
-    'oracledb>=1.0.0',
-]
-pagerduty = [
-    'pdpyras>=4.1.2',
-]
 pandas = [
-    pandas_requirement,
-]
-papermill = [
-    'papermill[all]>=1.2.1',
-    'scrapbook[all]',
+    'pandas>=0.17.1',
 ]
 password = [
     'bcrypt>=2.0.0',
     'flask-bcrypt>=0.7.1',
 ]
-pinot = [
-    # pinotdb v0.1.1 may still work with older versions of Apache Pinot, but we've confirmed that it
-    # causes a problem with newer versions.
-    'pinotdb>0.1.2',
-]
-plexus = [
-    'arrow>=0.16.0',
-]
-postgres = [
-    'psycopg2-binary>=2.7.4',
-]
-presto = [
-    'presto-python-client>=0.8.2',
-    pandas_requirement,
-]
-psrp = [
-    'pypsrp>=0.8',
-]
-qubole = [
-    'qds-sdk>=1.10.4',
-]
 rabbitmq = [
     'amqp',
 ]
-redis = [
-    # Redis 4 introduced a number of changes that likely need testing including mixins in redis commands
-    # as well as unquoting URLS with `urllib.parse.unquote`:
-    # https://github.com/redis/redis-py/blob/master/CHANGES
-    # TODO: upgrade to support redis package >=4
-    'redis~=3.2',
-]
-salesforce = ['simple-salesforce>=1.0.0', 'tableauserverclient', pandas_requirement]
-samba = [
-    'smbprotocol>=1.5.0',
-]
-segment = [
-    'analytics-python>=1.2.9',
-]
-sendgrid = [
-    'sendgrid>=6.0.0',
-]
 sentry = [
     'blinker>=1.1',
     'sentry-sdk>=0.8.0',
 ]
-singularity = ['spython>=0.0.56']
-slack = [
-    'slack_sdk>=3.0.0',
-]
-snowflake = [
-    'snowflake-connector-python>=2.4.1',
-    'snowflake-sqlalchemy>=1.1.0',
-]
-spark = [
-    'pyspark',
-]
-ssh = [
-    'paramiko>=2.6.0',
-    'sshtunnel>=0.3.2',
-]
 statsd = [
     'statsd>=3.3.0',
 ]
-tableau = [
-    'tableauserverclient',
-]
-telegram = [
-    'python-telegram-bot>=13.0',
-]
-trino = [
-    'trino>=0.301.0',
-    pandas_requirement,
-]
-vertica = [
-    'vertica-python>=0.5.1',
-]
 virtualenv = [
     'virtualenv',
 ]
 webhdfs = [
     'hdfs[avro,dataframe,kerberos]>=2.0.4',
 ]
-winrm = [
-    'pywinrm>=0.4',
-]
-yandex = [
-    'yandexcloud>=0.146.0',
-]
-zendesk = [
-    'zenpy>=2.0.24',
-]
 # End dependencies group
 
 # Mypy 0.900 and above ships only with stubs from stdlib so if we need other stubs, we need to install them
@@ -658,105 +399,59 @@ devel_only = [
     'yamllint',
 ]
 
-devel = cgroups + devel_only + doc + kubernetes + mypy_dependencies + mysql + pandas + password
-devel_hadoop = devel + hdfs + hive + kerberos + presto + webhdfs
-
-# Dict of all providers which are part of the Apache Airflow repository together with their requirements
-PROVIDERS_REQUIREMENTS: Dict[str, List[str]] = {
-    'airbyte': http_provider,
-    'alibaba': alibaba,
-    'amazon': amazon,
-    'apache.beam': apache_beam,
-    'apache.cassandra': cassandra,
-    'apache.drill': drill,
-    'apache.druid': druid,
-    'apache.hdfs': hdfs,
-    'apache.hive': hive,
-    'apache.kylin': kylin,
-    'apache.livy': http_provider,
-    'apache.pig': [],
-    'apache.pinot': pinot,
-    'apache.spark': spark,
-    'apache.sqoop': [],
-    'arangodb': arangodb,
-    'asana': asana,
-    'celery': celery,
-    'cloudant': cloudant,
-    'cncf.kubernetes': kubernetes,
-    'core.sql': [],
-    'databricks': databricks,
-    'datadog': datadog,
-    'dbt.cloud': http_provider,
-    'dingding': [],
-    'discord': [],
-    'docker': docker,
-    'elasticsearch': elasticsearch,
-    'exasol': exasol,
-    'facebook': facebook,
-    'ftp': [],
-    'github': github,
-    'google': google,
-    'grpc': grpc,
-    'hashicorp': hashicorp,
-    'http': http,
-    'imap': [],
-    'influxdb': influxdb,
-    'jdbc': jdbc,
-    'jenkins': jenkins,
-    'jira': jira,
-    'microsoft.azure': azure,
-    'microsoft.mssql': mssql,
-    'microsoft.psrp': psrp,
-    'microsoft.winrm': winrm,
-    'mongo': mongo,
-    'mysql': mysql,
-    'neo4j': neo4j,
-    'odbc': odbc,
-    'openfaas': [],
-    'opsgenie': opsgenie,
-    'oracle': oracle,
-    'pagerduty': pagerduty,
-    'papermill': papermill,
-    'plexus': plexus,
-    'postgres': postgres,
-    'presto': presto,
-    'qubole': qubole,
-    'redis': redis,
-    'salesforce': salesforce,
-    'samba': samba,
-    'segment': segment,
-    'sendgrid': sendgrid,
-    'sftp': ssh,
-    'singularity': singularity,
-    'slack': slack,
-    'snowflake': snowflake,
-    'sqlite': [],
-    'ssh': ssh,
-    'tableau': tableau,
-    'telegram': telegram,
-    'trino': trino,
-    'vertica': vertica,
-    'yandex': yandex,
-    'zendesk': zendesk,
-}
+
+def get_provider_dependencies(provider_name: str) -> List[str]:
+    return PROVIDER_DEPENDENCIES[provider_name][DEPS]
+
+
+def get_unique_dependency_list(req_list_iterable: Iterable[List[str]]):
+    _all_reqs: Set[str] = set()
+    for req_list in req_list_iterable:
+        for req in req_list:
+            _all_reqs.add(req)
+    return list(_all_reqs)
+
+
+devel = get_unique_dependency_list(
+    [
+        cgroups,
+        devel_only,
+        doc,
+        kubernetes,
+        mypy_dependencies,
+        get_provider_dependencies('mysql'),
+        pandas,
+        password,
+    ]
+)
+
+devel_hadoop = get_unique_dependency_list(
+    [
+        devel,
+        get_provider_dependencies('apache.hdfs'),
+        get_provider_dependencies('apache.hive'),
+        kerberos,
+        get_provider_dependencies('presto'),
+        webhdfs,
+    ]
+)
 
 # Those are all additional extras which do not have their own 'providers'
 # The 'apache.atlas' and 'apache.webhdfs' are extras that provide additional libraries
 # but they do not have separate providers (yet?), they are merely there to add extra libraries
 # That can be used in custom python/bash operators.
-ADDITIONAL_EXTRAS_REQUIREMENTS: Dict[str, List[str]] = {
+ADDITIONAL_EXTRAS_DEPENDENCIES: Dict[str, List[str]] = {
     'apache.atlas': atlas,
     'apache.webhdfs': webhdfs,
 }
 
-
 # Those are extras that are extensions of the 'core' Airflow. They provide additional features
 # To airflow core. They do not have separate providers because they do not have any operators/hooks etc.
-CORE_EXTRAS_REQUIREMENTS: Dict[str, List[str]] = {
+CORE_EXTRAS_DEPENDENCIES: Dict[str, List[str]] = {
     'async': async_packages,
-    'celery': celery,  # also has provider, but it extends the core with the CeleryExecutor
+    'celery': celery,
     'cgroups': cgroups,
-    'cncf.kubernetes': kubernetes,  # also has provider, but it extends the core with the KubernetesExecutor
+    'cncf.kubernetes': kubernetes,
     'dask': dask,
     'deprecated_api': deprecated_api,
     'github_enterprise': flask_appbuilder_oauth,
@@ -772,23 +467,17 @@ CORE_EXTRAS_REQUIREMENTS: Dict[str, List[str]] = {
     'virtualenv': virtualenv,
 }
 
-EXTRAS_REQUIREMENTS: Dict[str, List[str]] = deepcopy(CORE_EXTRAS_REQUIREMENTS)
+EXTRAS_DEPENDENCIES: Dict[str, List[str]] = deepcopy(CORE_EXTRAS_DEPENDENCIES)
 
 
 def add_extras_for_all_providers() -> None:
-    """
-    Adds extras for all providers.
-    By default all providers have the same extra name as provider id, for example
-    'apache.hive' extra has 'apache.hive' provider requirement.
-    """
-    for provider_name, provider_requirement in PROVIDERS_REQUIREMENTS.items():
-        EXTRAS_REQUIREMENTS[provider_name] = provider_requirement
+    for (provider_name, provider_dict) in PROVIDER_DEPENDENCIES.items():
+        EXTRAS_DEPENDENCIES[provider_name] = provider_dict[DEPS]
 
 
 def add_additional_extras() -> None:
-    """Adds extras for all additional extras."""
-    for extra_name, extra_requirement in ADDITIONAL_EXTRAS_REQUIREMENTS.items():
-        EXTRAS_REQUIREMENTS[extra_name] = extra_requirement
+    for (extra_name, extra_dependencies) in ADDITIONAL_EXTRAS_DEPENDENCIES.items():
+        EXTRAS_DEPENDENCIES[extra_name] = extra_dependencies
 
 
 add_extras_for_all_providers()
@@ -804,7 +493,7 @@ EXTRAS_DEPRECATED_ALIASES: Dict[str, str] = {
     'aws': 'amazon',
     'azure': 'microsoft.azure',
     'cassandra': 'apache.cassandra',
-    'crypto': '',  # All crypto requirements are installation requirements of core Airflow
+    'crypto': '',  # this is legacy extra - all dependencies are already "install-requires"
     'druid': 'apache.druid',
     'gcp': 'google',
     'gcp_api': 'google',
@@ -830,19 +519,19 @@ def add_extras_for_all_deprecated_aliases() -> None:
     """
     Add extras for all deprecated aliases. Requirements for those deprecated aliases are the same
     as the extras they are replaced with.
-    The requirements are not copies - those are the same lists as for the new extras. This is intended.
+    The dependencies are not copies - those are the same lists as for the new extras. This is intended.
     Thanks to that if the original extras are later extended with providers, aliases are extended as well.
     """
     for alias, extra in EXTRAS_DEPRECATED_ALIASES.items():
-        requirements = EXTRAS_REQUIREMENTS.get(extra) if extra != '' else []
-        if requirements is None:
+        dependencies = EXTRAS_DEPENDENCIES.get(extra) if extra != '' else []
+        if dependencies is None:
             raise Exception(f"The extra {extra} is missing for deprecated alias {alias}")
-        EXTRAS_REQUIREMENTS[alias] = requirements
+        EXTRAS_DEPENDENCIES[alias] = dependencies
 
 
 def add_all_deprecated_provider_packages() -> None:
     """
-    For deprecated aliases that are providers, we will swap the providers requirements to instead
+    For deprecated aliases that are providers, we will swap the providers dependencies to instead
     be the provider itself.
 
     e.g. {"kubernetes": ["kubernetes>=3.0.0, <12.0.0", ...]} becomes
@@ -851,7 +540,7 @@ def add_all_deprecated_provider_packages() -> None:
     for alias, provider in EXTRAS_DEPRECATED_ALIASES.items():
         if alias in EXTRAS_DEPRECATED_ALIASES_NOT_PROVIDERS:
             continue
-        replace_extra_requirement_with_provider_packages(alias, [provider])
+        replace_extra_dependencies_with_provider_packages(alias, [provider])
 
 
 add_extras_for_all_deprecated_aliases()
@@ -862,7 +551,7 @@ add_extras_for_all_deprecated_aliases()
 
 # This is list of all providers. It's a shortcut for anyone who would like to easily get list of
 # All providers. It is used by pre-commits.
-ALL_PROVIDERS = list(PROVIDERS_REQUIREMENTS.keys())
+ALL_PROVIDERS = list(PROVIDER_DEPENDENCIES.keys())
 
 ALL_DB_PROVIDERS = [
     'apache.cassandra',
@@ -886,30 +575,35 @@ ALL_DB_PROVIDERS = [
     'vertica',
 ]
 
-# Special requirements for all database-related providers. They are de-duplicated.
-all_dbs = list({req for db_provider in ALL_DB_PROVIDERS for req in PROVIDERS_REQUIREMENTS[db_provider]})
 
-# Requirements for all "user" extras (no devel). They are de-duplicated. Note that we do not need
-# to separately add providers requirements - they have been already added as 'providers' extras above
-_all_requirements = list({req for extras_reqs in EXTRAS_REQUIREMENTS.values() for req in extras_reqs})
+def get_all_db_dependencies() -> List[str]:
+    _all_db_reqs: Set[str] = set()
+    for provider in ALL_DB_PROVIDERS:
+        for req in PROVIDER_DEPENDENCIES[provider][DEPS]:
+            _all_db_reqs.add(req)
+    return list(_all_db_reqs)
 
-# All user extras here
-EXTRAS_REQUIREMENTS["all"] = _all_requirements
+
+# Special dependencies for all database-related providers. They are de-duplicated.
+all_dbs = get_all_db_dependencies()
 
 # All db user extras here
-EXTRAS_REQUIREMENTS["all_dbs"] = all_dbs + pandas
+EXTRAS_DEPENDENCIES["all_dbs"] = all_dbs
+
+# Requirements for all "user" extras (no devel). They are de-duplicated. Note that we do not need
+# to separately add providers dependencies - they have been already added as 'providers' extras above
+_all_dependencies = get_unique_dependency_list(EXTRAS_DEPENDENCIES.values())
+
+# All user extras here
+EXTRAS_DEPENDENCIES["all"] = _all_dependencies
 
-# This can be simplified to devel_hadoop + _all_requirements due to inclusions
+# This can be simplified to devel_hadoop + _all_dependencies due to inclusions
 # but we keep it for explicit sake. We are de-duplicating it anyway.
-devel_all = list(set(_all_requirements + doc + devel + devel_hadoop))
+devel_all = get_unique_dependency_list([_all_dependencies, doc, devel, devel_hadoop])
 
 # Those are packages excluded for "all" dependencies
 PACKAGES_EXCLUDED_FOR_ALL = []
-PACKAGES_EXCLUDED_FOR_ALL.extend(
-    [
-        'snakebite',
-    ]
-)
+PACKAGES_EXCLUDED_FOR_ALL.extend(['snakebite'])
 
 
 def is_package_excluded(package: str, exclusion_list: List[str]) -> bool:
@@ -934,26 +628,27 @@ devel_ci = devel_all
 
 # Those are extras that we have to add for development purposes
 # They can be use to install some predefined set of dependencies.
-EXTRAS_REQUIREMENTS["doc"] = doc
-EXTRAS_REQUIREMENTS["devel"] = devel  # devel already includes doc
-EXTRAS_REQUIREMENTS["devel_hadoop"] = devel_hadoop  # devel_hadoop already includes devel
-EXTRAS_REQUIREMENTS["devel_all"] = devel_all
-EXTRAS_REQUIREMENTS["devel_ci"] = devel_ci
+EXTRAS_DEPENDENCIES["doc"] = doc
+EXTRAS_DEPENDENCIES["devel"] = devel  # devel already includes doc
+EXTRAS_DEPENDENCIES["devel_hadoop"] = devel_hadoop  # devel_hadoop already includes devel
+EXTRAS_DEPENDENCIES["devel_all"] = devel_all
+EXTRAS_DEPENDENCIES["devel_ci"] = devel_ci
 
 
-def sort_extras_requirements() -> Dict[str, List[str]]:
+def sort_extras_dependencies() -> Dict[str, List[str]]:
     """
     The dictionary order remains when keys() are retrieved.
     Sort both: extras and list of dependencies to make it easier to analyse problems
     external packages will be first, then if providers are added they are added at the end of the lists.
     """
-    sorted_requirements = dict(sorted(EXTRAS_REQUIREMENTS.items()))
-    for extra_list in sorted_requirements.values():
-        extra_list.sort()
-    return sorted_requirements
+    sorted_dependencies: Dict[str, List[str]] = {}
+    sorted_extra_ids = sorted(EXTRAS_DEPENDENCIES.keys())
+    for extra_id in sorted_extra_ids:
+        sorted_dependencies[extra_id] = sorted(EXTRAS_DEPENDENCIES[extra_id])
+    return sorted_dependencies
 
 
-EXTRAS_REQUIREMENTS = sort_extras_requirements()
+EXTRAS_DEPENDENCIES = sort_extras_dependencies()
 
 # Those providers are pre-installed always when airflow is installed.
 # Those providers do not have dependency on airflow2.0 because that would lead to circular dependencies.
@@ -966,7 +661,7 @@ PREINSTALLED_PROVIDERS = [
 ]
 
 
-def get_provider_package_from_package_id(package_id: str) -> str:
+def get_provider_package_name_from_package_id(package_id: str) -> str:
     """
     Builds the name of provider package out of the package id provided/
 
@@ -986,8 +681,8 @@ def get_all_provider_packages() -> str:
     """Returns all provider packages configured in setup.py"""
     excluded_providers = get_excluded_providers()
     return " ".join(
-        get_provider_package_from_package_id(package)
-        for package in PROVIDERS_REQUIREMENTS
+        get_provider_package_name_from_package_id(package)
+        for package in ALL_PROVIDERS
         if package not in excluded_providers
     )
 
@@ -1013,27 +708,30 @@ class AirflowDistribution(Distribution):
             ]
             provider_yaml_files = glob.glob("airflow/providers/**/provider.yaml", recursive=True)
             for provider_yaml_file in provider_yaml_files:
-                provider_relative_path = relpath(provider_yaml_file, os.path.join(my_dir, "airflow"))
+                provider_relative_path = relpath(provider_yaml_file, str(AIRFLOW_SOURCES_ROOT / "airflow"))
                 self.package_data['airflow'].append(provider_relative_path)
         else:
             self.install_requires.extend(
-                [get_provider_package_from_package_id(package_id) for package_id in PREINSTALLED_PROVIDERS]
+                [
+                    get_provider_package_name_from_package_id(package_id)
+                    for package_id in PREINSTALLED_PROVIDERS
+                ]
             )
 
 
-def replace_extra_requirement_with_provider_packages(extra: str, providers: List[str]) -> None:
+def replace_extra_dependencies_with_provider_packages(extra: str, providers: List[str]) -> None:
     """
-    Replaces extra requirement with provider package. The intention here is that when
+    Replaces extra dependencies with provider package. The intention here is that when
     the provider is added as dependency of extra, there is no need to add the dependencies
     separately. This is not needed and even harmful, because in case of future versions of
-    the provider, the requirements might change, so hard-coding requirements from the version
+    the provider, the dependencies might change, so hard-coding dependencies from the version
     that was available at the release time might cause dependency conflicts in the future.
 
     Say for example that you have salesforce provider with those deps:
 
     { 'salesforce': ['simple-salesforce>=1.0.0', 'tableauserverclient'] }
 
-    Initially ['salesforce'] extra has those requirements and it works like that when you install
+    Initially ['salesforce'] extra has those dependencies, and it works like that when you install
     it when INSTALL_PROVIDERS_FROM_SOURCES is set to `true` (during the development). However, when
     the production installation is used, The dependencies are changed:
 
@@ -1042,7 +740,7 @@ def replace_extra_requirement_with_provider_packages(extra: str, providers: List
     And then, 'apache-airflow-providers-salesforce' package has those 'install_requires' dependencies:
             ['simple-salesforce>=1.0.0', 'tableauserverclient']
 
-    So transitively 'salesforce' extra has all the requirements it needs and in case the provider
+    So transitively 'salesforce' extra has all the dependencies it needs and in case the provider
     changes its dependencies, they will transitively change as well.
 
     In the constraint mechanism we save both - provider versions and it's dependencies
@@ -1059,26 +757,26 @@ def replace_extra_requirement_with_provider_packages(extra: str, providers: List
     :param providers: list of provider ids
     """
     if extra in ['cncf.kubernetes', 'kubernetes', 'celery']:
-        EXTRAS_REQUIREMENTS[extra].extend(
-            [get_provider_package_from_package_id(package_name) for package_name in providers]
+        EXTRAS_DEPENDENCIES[extra].extend(
+            [get_provider_package_name_from_package_id(package_name) for package_name in providers]
         )
     else:
-        EXTRAS_REQUIREMENTS[extra] = [
-            get_provider_package_from_package_id(package_name) for package_name in providers
+        EXTRAS_DEPENDENCIES[extra] = [
+            get_provider_package_name_from_package_id(package_name) for package_name in providers
         ]
 
 
-def add_provider_packages_to_extra_requirements(extra: str, providers: List[str]) -> None:
+def add_provider_packages_to_extra_dependencies(extra: str, providers: List[str]) -> None:
     """
-    Adds provider packages as requirements to extra. This is used to add provider packages as requirements
-    to the "bulk" kind of extras. Those bulk extras do not have the detailed 'extra' requirements as
+    Adds provider packages as dependencies to extra. This is used to add provider packages as dependencies
+    to the "bulk" kind of extras. Those bulk extras do not have the detailed 'extra' dependencies as
     initial values, so instead of replacing them (see previous function) we can extend them.
 
     :param extra: Name of the extra to add providers to
     :param providers: list of provider ids
     """
-    EXTRAS_REQUIREMENTS[extra].extend(
-        [get_provider_package_from_package_id(package_name) for package_name in providers]
+    EXTRAS_DEPENDENCIES[extra].extend(
+        [get_provider_package_name_from_package_id(package_name) for package_name in providers]
     )
 
 
@@ -1092,13 +790,13 @@ def add_all_provider_packages() -> None:
     as the new provider is not yet in PyPI.
 
     """
-    for provider in ALL_PROVIDERS:
-        replace_extra_requirement_with_provider_packages(provider, [provider])
-    add_provider_packages_to_extra_requirements("all", ALL_PROVIDERS)
-    add_provider_packages_to_extra_requirements("devel_ci", ALL_PROVIDERS)
-    add_provider_packages_to_extra_requirements("devel_all", ALL_PROVIDERS)
-    add_provider_packages_to_extra_requirements("all_dbs", ALL_DB_PROVIDERS)
-    add_provider_packages_to_extra_requirements(
+    for provider_id in ALL_PROVIDERS:
+        replace_extra_dependencies_with_provider_packages(provider_id, [provider_id])
+    add_provider_packages_to_extra_dependencies("all", ALL_PROVIDERS)
+    add_provider_packages_to_extra_dependencies("devel_ci", ALL_PROVIDERS)
+    add_provider_packages_to_extra_dependencies("devel_all", ALL_PROVIDERS)
+    add_provider_packages_to_extra_dependencies("all_dbs", ALL_DB_PROVIDERS)
+    add_provider_packages_to_extra_dependencies(
         "devel_hadoop", ["apache.hdfs", "apache.hive", "presto", "trino"]
     )
     add_all_deprecated_provider_packages()
@@ -1164,7 +862,7 @@ def do_setup() -> None:
     setup(
         distclass=AirflowDistribution,
         version=version,
-        extras_require=EXTRAS_REQUIREMENTS,
+        extras_require=EXTRAS_DEPENDENCIES,
         download_url=('https://archive.apache.org/dist/airflow/' + version),
         cmdclass={
             'extra_clean': CleanCommand,
diff --git a/tests/build_provider_packages_dependencies.py b/tests/build_provider_packages_dependencies.py
deleted file mode 100644
index 14e3e3fca7..0000000000
--- a/tests/build_provider_packages_dependencies.py
+++ /dev/null
@@ -1,280 +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.
-import json
-import os
-import sys
-from ast import Import, ImportFrom, NodeVisitor, parse
-from collections import defaultdict
-from os.path import dirname, sep
-from typing import Dict, List, Optional, Tuple
-
-from setup import PROVIDERS_REQUIREMENTS
-
-sys.path.append(os.path.join(dirname(__file__), os.pardir))
-
-
-AIRFLOW_PROVIDERS_FILE_PREFIX = f"airflow{sep}providers{sep}"
-AIRFLOW_TESTS_PROVIDERS_FILE_PREFIX = f"tests{sep}providers{sep}"
-AIRFLOW_PROVIDERS_IMPORT_PREFIX = "airflow.providers."
-
-# List of information messages generated
-infos: List[str] = []
-# List of warnings generated
-warnings: List[str] = []
-# list of errors generated
-errors: List[str] = []
-
-# store dependencies
-dependencies: Dict[str, List[str]] = defaultdict(list)
-
-
-def find_provider(provider_elements: List[str]) -> Optional[str]:
-    """
-    Finds provider name from the list of elements provided. It looks the providers up
-    in PROVIDERS_REQUIREMENTS dict taken from the setup.py.
-
-    :param provider_elements: array of elements of the path (split)
-    :return: provider name or None if no provider could be found
-    """
-    provider = ""
-    separator = ""
-    provider_keys = PROVIDERS_REQUIREMENTS.keys()
-    for element in provider_elements:
-        provider = provider + separator + element
-        if provider in provider_keys:
-            return provider
-        separator = "."
-    return None
-
-
-def get_provider_from_file_name(file_name: str) -> Optional[str]:
-    """
-    Retrieves provider name from file name
-    :param file_name: name of the file
-    :return: provider name or None if no provider could be found
-    """
-    if (
-        AIRFLOW_PROVIDERS_FILE_PREFIX not in file_name
-        and AIRFLOW_TESTS_PROVIDERS_FILE_PREFIX not in file_name
-    ):
-        # We should only check file that are provider
-        errors.append(f"Wrong file not in the providers package = {file_name}")
-        return None
-    suffix = get_file_suffix(file_name)
-    assert suffix
-    split_path = suffix.split(sep)[2:]
-    provider = find_provider(split_path)
-    if not provider and file_name.endswith("__init__.py"):
-        infos.append(f"Skipped file = {file_name}")
-    elif not provider:
-        warnings.append(f"Provider not found for path = {file_name}")
-    return provider
-
-
-def get_file_suffix(file_name) -> Optional[str]:
-    if AIRFLOW_PROVIDERS_FILE_PREFIX in file_name:
-        return file_name[file_name.find(AIRFLOW_PROVIDERS_FILE_PREFIX) :]
-    if AIRFLOW_TESTS_PROVIDERS_FILE_PREFIX in file_name:
-        return file_name[file_name.find(AIRFLOW_TESTS_PROVIDERS_FILE_PREFIX) :]
-    return None
-
-
-def get_provider_from_import(import_name: str) -> Optional[str]:
-    """
-    Retrieves provider name from file name
-    :param import_name: name of the import
-    :return: provider name or None if no provider could be found
-    """
-    if AIRFLOW_PROVIDERS_IMPORT_PREFIX not in import_name:
-        # skip silently - we expect non-providers imports
-        return None
-    suffix = import_name[import_name.find(AIRFLOW_PROVIDERS_IMPORT_PREFIX) :]
-    split_import = suffix.split(".")[2:]
-    provider = find_provider(split_import)
-    if not provider:
-        warnings.append(f"Provider not found for import = {import_name}")
-    return provider
-
-
-class ImportFinder(NodeVisitor):
-    """
-    AST visitor that collects all imported names in its imports
-    """
-
-    def __init__(self, filename: str) -> None:
-        self.imports: List[str] = []
-        self.filename = filename
-        self.handled_import_exception = List[str]
-        self.tried_imports: List[str] = []
-
-    def process_import(self, import_name: str) -> None:
-        self.imports.append(import_name)
-
-    def get_import_name_from_import_from(self, node: ImportFrom) -> List[str]:
-        """
-        Retrieves import name from the "from" import.
-        :param node: ImportFrom name
-        :return: import name
-        """
-        import_names: List[str] = []
-        for alias in node.names:
-            name = alias.name
-            fullname = f'{node.module}.{name}' if node.module else name
-            import_names.append(fullname)
-        return import_names
-
-    def visit_Import(self, node: Import):
-        for alias in node.names:
-            self.process_import(alias.name)
-
-    def visit_ImportFrom(self, node: ImportFrom):
-        if node.module == '__future__':
-            return
-        for fullname in self.get_import_name_from_import_from(node):
-            self.process_import(fullname)
-
-
-def get_imports_from_file(file_name: str) -> List[str]:
-    """
-    Retrieves imports from file.
-    :param file_name: name of the file
-    :return: list of import names
-    """
-    try:
-        with open(file_name, encoding="utf-8") as f:
-            root = parse(f.read(), file_name)
-    except Exception:
-        print(f"Error when opening file {file_name}", file=sys.stderr)
-        raise
-    visitor = ImportFinder(file_name)
-    visitor.visit(root)
-    return visitor.imports
-
-
-def check_if_different_provider_used(file_name: str) -> None:
-    file_provider = get_provider_from_file_name(file_name)
-    if not file_provider:
-        return
-    imports = get_imports_from_file(file_name)
-    for import_name in imports:
-        import_provider = get_provider_from_import(import_name)
-        if import_provider and file_provider != import_provider:
-            dependencies[file_provider].append(import_provider)
-
-
-def parse_arguments() -> Tuple[str, str, str]:
-    import argparse
-
-    parser = argparse.ArgumentParser(
-        description='Checks if dependencies between packages are handled correctly.'
-    )
-    parser.add_argument(
-        "-f", "--provider-dependencies-file", help="Stores dependencies between providers in the file(.json)"
-    )
-    parser.add_argument(
-        "-d", "--documentation-file", help="Updates package documentation in the file specified (.rst)"
-    )
-    parser.add_argument('files', nargs='*')
-    args = parser.parse_args()
-
-    if len(args.files) < 1:
-        parser.print_usage()
-        print()
-        sys.exit(2)
-    return args.files, args.provider_dependencies_file, args.documentation_file
-
-
-PREFIX = "    "
-
-HEADER = """
-========================== ===========================
-Package                    Extras
-========================== ===========================
-"""
-FOOTER = """========================== ===========================
-
-"""
-
-
-def insert_documentation(deps_dict: Dict[str, List[str]], res: List[str]) -> None:
-    res += HEADER.splitlines(keepends=True)
-    for package, deps in deps_dict.items():
-        deps_str = ",".join(deps)
-        res.append(f"{package:27}{deps_str}\n")
-    res += FOOTER.splitlines(keepends=True)
-
-
-if __name__ == '__main__':
-    print()
-    files, provider_dependencies_file_name, documentation_file_name = parse_arguments()
-    num_files = 0
-    for file in files:
-        check_if_different_provider_used(file)
-        num_files += 1
-    print(f"Verified {num_files} files.")
-    if infos:
-        print("\nInformation messages:\n")
-        for info in infos:
-            print(PREFIX + info)
-        print(f"Total: {len(infos)} information messages.")
-    if warnings:
-        print("\nWarnings!\n")
-        for warning in warnings:
-            print(PREFIX + warning)
-        print(f"Total: {len(warnings)} warnings.")
-    if errors:
-        print("\nErrors!\n")
-        for error in errors:
-            print(PREFIX + error)
-        print(f"Total: {len(errors)} errors.")
-    unique_sorted_dependencies: Dict[str, List[str]] = {}
-    for key in sorted(dependencies.keys()):
-        unique_sorted_dependencies[key] = sorted(set(dependencies[key]))
-    if provider_dependencies_file_name:
-        with open(provider_dependencies_file_name, "w") as providers_file:
-            json.dump(unique_sorted_dependencies, providers_file, indent=2)
-            providers_file.write("\n")
-        print()
-        print(f"Written provider dependencies to the file {provider_dependencies_file_name}")
-        print()
-    if documentation_file_name:
-        with open(documentation_file_name, encoding="utf-8") as documentation_file:
-            text = documentation_file.readlines()
-        replacing = False
-        result: List[str] = []
-        for line in text:
-            if line.startswith("  .. START PACKAGE DEPENDENCIES HERE"):
-                replacing = True
-                result.append(line)
-                insert_documentation(unique_sorted_dependencies, result)
-            if line.startswith("  .. END PACKAGE DEPENDENCIES HERE"):
-                replacing = False
-            if not replacing:
-                result.append(line)
-        with open(documentation_file_name, "w", encoding="utf-8") as documentation_file:
-            documentation_file.write("".join(result))
-        print()
-        print(f"Written package extras to the file {documentation_file_name}")
-        print()
-    if errors:
-        print()
-        print("ERROR! Errors found during verification. Exiting!")
-        print()
-        sys.exit(1)
-    print()
-    print("Verification complete! Success!")
-    print()