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/07/06 22:02:24 UTC

[airflow] branch main updated: Move all SQL classes to common-sql provider (#24836)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 46bbfdade0 Move all SQL classes to common-sql provider (#24836)
46bbfdade0 is described below

commit 46bbfdade0638cb8a5d187e47034b84e68ddf762
Author: Jarek Potiuk <ja...@polidea.com>
AuthorDate: Thu Jul 7 00:02:16 2022 +0200

    Move all SQL classes to common-sql provider (#24836)
    
    The DBApiHook, SQLSensor are now part of the common.sql provider.
---
 .../airflow_providers_bug_report.yml               |   2 +-
 CONTRIBUTING.rst                                   |  18 +-
 INSTALL                                            |  18 +-
 airflow/hooks/dbapi.py                             | 370 +--------------------
 airflow/hooks/dbapi_hook.py                        |   6 +-
 airflow/operators/sql.py                           |   2 +-
 airflow/providers/amazon/aws/hooks/redshift_sql.py |   2 +-
 .../providers/amazon/aws/transfers/sql_to_s3.py    |   2 +-
 airflow/providers/amazon/provider.yaml             |   1 +
 airflow/providers/apache/drill/hooks/drill.py      |   2 +-
 airflow/providers/apache/drill/provider.yaml       |   1 +
 airflow/providers/apache/druid/hooks/druid.py      |   2 +-
 airflow/providers/apache/druid/provider.yaml       |   1 +
 airflow/providers/apache/hive/hooks/hive.py        |   2 +-
 airflow/providers/apache/hive/provider.yaml        |   1 +
 .../apache/hive/sensors/metastore_partition.py     |   2 +-
 airflow/providers/apache/pinot/hooks/pinot.py      |   2 +-
 airflow/providers/apache/pinot/provider.yaml       |   1 +
 airflow/providers/{core => common}/__init__.py     |   0
 .../providers/{core => common}/sql/CHANGELOG.rst   |   3 +-
 airflow/providers/{core => common}/sql/__init__.py |   0
 .../example_dags => common/sql/hooks}/__init__.py  |   0
 .../dbapi.py => providers/common/sql/hooks/sql.py} |  47 ++-
 .../{core => common}/sql/operators/__init__.py     |   0
 .../{core => common}/sql/operators/sql.py          |   4 +-
 .../providers/{core => common}/sql/provider.yaml   |  32 +-
 .../providers/common/sql/sensors}/__init__.py      |   0
 airflow/{ => providers/common/sql}/sensors/sql.py  |  21 +-
 .../providers/databricks/hooks/databricks_sql.py   |   2 +-
 airflow/providers/databricks/provider.yaml         |   1 +
 .../providers/elasticsearch/hooks/elasticsearch.py |   2 +-
 airflow/providers/elasticsearch/provider.yaml      |   1 +
 airflow/providers/exasol/hooks/exasol.py           |   2 +-
 airflow/providers/exasol/provider.yaml             |   1 +
 airflow/providers/google/cloud/hooks/bigquery.py   |   2 +-
 airflow/providers/google/provider.yaml             |   1 +
 airflow/providers/jdbc/hooks/jdbc.py               |   2 +-
 airflow/providers/jdbc/provider.yaml               |   1 +
 airflow/providers/microsoft/mssql/hooks/mssql.py   |   2 +-
 .../providers/microsoft/mssql/operators/mssql.py   |   2 +-
 airflow/providers/microsoft/mssql/provider.yaml    |   1 +
 airflow/providers/mysql/hooks/mysql.py             |   2 +-
 airflow/providers/mysql/provider.yaml              |   1 +
 airflow/providers/odbc/hooks/odbc.py               |   7 +-
 airflow/providers/odbc/provider.yaml               |   1 +
 airflow/providers/oracle/hooks/oracle.py           |   2 +-
 airflow/providers/oracle/provider.yaml             |   1 +
 airflow/providers/postgres/hooks/postgres.py       |   2 +-
 airflow/providers/postgres/provider.yaml           |   1 +
 airflow/providers/presto/hooks/presto.py           |   2 +-
 airflow/providers/presto/provider.yaml             |   1 +
 airflow/providers/qubole/hooks/qubole_check.py     |   2 +-
 airflow/providers/qubole/provider.yaml             |   1 +
 airflow/providers/slack/provider.yaml              |   1 +
 airflow/providers/slack/transfers/sql_to_slack.py  |  32 +-
 airflow/providers/snowflake/hooks/snowflake.py     |   2 +-
 airflow/providers/snowflake/provider.yaml          |   1 +
 airflow/providers/sqlite/hooks/sqlite.py           |   2 +-
 airflow/providers/sqlite/provider.yaml             |   3 +-
 airflow/providers/trino/hooks/trino.py             |   2 +-
 airflow/providers/trino/provider.yaml              |   1 +
 airflow/providers/vertica/hooks/vertica.py         |   2 +-
 airflow/providers/vertica/provider.yaml            |   1 +
 airflow/sensors/sql.py                             |  99 +-----
 airflow/sensors/sql_sensor.py                      |   6 +-
 .../commits.rst                                    |   8 +-
 .../connections.rst                                |   0
 .../index.rst                                      |  20 +-
 .../installing-providers-from-sources.rst          |   0
 .../operators.rst                                  |  10 +-
 .../connections/odbc.rst                           |   2 +-
 docs/apache-airflow/extra-packages-ref.rst         |   2 +-
 generated/provider_dependencies.json               |  93 ++++--
 images/breeze/output-build-docs.svg                | 248 +++++++-------
 images/breeze/output-commands-hash.txt             |   6 +-
 .../output-prepare-provider-documentation.svg      | 132 ++++----
 images/breeze/output-prepare-provider-packages.svg | 136 ++++----
 newsfragments/NEW.significant.rst                  |   1 +
 scripts/ci/installed_providers.txt                 |   1 +
 setup.py                                           |   1 +
 tests/deprecated_classes.py                        |   4 -
 tests/operators/test_generic_transfer.py           |   4 +-
 tests/providers/{core/sql => common}/__init__.py   |   0
 .../{core/sql/operators => common/sql}/__init__.py |   0
 .../providers/common/sql/hooks}/__init__.py        |   0
 .../{ => providers/common/sql}/hooks/test_dbapi.py |   2 +-
 .../providers/common/sql/operators}/__init__.py    |   0
 .../{core => common}/sql/operators/test_sql.py     |   2 +-
 .../providers/common/sql/sensors}/__init__.py      |   0
 .../common/sql/sensors/test_sql.py}                |  19 +-
 .../providers/microsoft/mssql/hooks/test_mssql.py  |   6 +-
 tests/providers/presto/hooks/test_presto.py        |   2 +-
 tests/providers/trino/hooks/test_trino.py          |   2 +-
 tests/providers/vertica/hooks/test_vertica.py      |   2 +-
 .../system/providers/common}/__init__.py           |   0
 .../system/providers/common/sql}/__init__.py       |   0
 .../common/sql}/example_sql_column_table_check.py  |   8 +-
 97 files changed, 561 insertions(+), 887 deletions(-)

diff --git a/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml b/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml
index 268653ffa5..cd3958cdff 100644
--- a/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml
+++ b/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml
@@ -43,7 +43,7 @@ body:
         - celery
         - cloudant
         - cncf-kubernetes
-        - core-sql
+        - common-sql
         - databricks
         - datadog
         - dbt-cloud
diff --git a/CONTRIBUTING.rst b/CONTRIBUTING.rst
index 92b98dfca1..323172ad31 100644
--- a/CONTRIBUTING.rst
+++ b/CONTRIBUTING.rst
@@ -620,15 +620,15 @@ This is the full list of those extras:
 airbyte, alibaba, all, all_dbs, amazon, apache.atlas, apache.beam, apache.cassandra, apache.drill,
 apache.druid, apache.hdfs, apache.hive, apache.kylin, apache.livy, apache.pig, apache.pinot,
 apache.spark, apache.sqoop, apache.webhdfs, arangodb, asana, async, atlas, aws, azure, cassandra,
-celery, cgroups, cloudant, cncf.kubernetes, core.sql, crypto, dask, databricks, datadog, dbt.cloud,
-deprecated_api, devel, devel_all, devel_ci, devel_hadoop, dingding, discord, doc, docker, druid,
-elasticsearch, exasol, facebook, ftp, gcp, gcp_api, github, github_enterprise, google, google_auth,
-grpc, hashicorp, hdfs, hive, http, imap, influxdb, jdbc, jenkins, jira, kerberos, kubernetes, ldap,
-leveldb, microsoft.azure, microsoft.mssql, microsoft.psrp, microsoft.winrm, mongo, mssql, mysql,
-neo4j, odbc, openfaas, opsgenie, oracle, pagerduty, pandas, papermill, password, pinot, plexus,
-postgres, presto, qds, qubole, rabbitmq, redis, s3, salesforce, samba, segment, sendgrid, sentry,
-sftp, singularity, slack, snowflake, spark, sqlite, ssh, statsd, tableau, tabular, telegram, trino,
-vertica, virtualenv, webhdfs, winrm, yandex, zendesk
+celery, cgroups, cloudant, cncf.kubernetes, common.sql, crypto, dask, databricks, datadog,
+dbt.cloud, deprecated_api, devel, devel_all, devel_ci, devel_hadoop, dingding, discord, doc, docker,
+druid, elasticsearch, exasol, facebook, ftp, gcp, gcp_api, github, github_enterprise, google,
+google_auth, grpc, hashicorp, hdfs, hive, http, imap, influxdb, jdbc, jenkins, jira, kerberos,
+kubernetes, ldap, leveldb, microsoft.azure, microsoft.mssql, microsoft.psrp, microsoft.winrm, mongo,
+mssql, mysql, neo4j, odbc, openfaas, opsgenie, oracle, pagerduty, pandas, papermill, password,
+pinot, plexus, postgres, presto, qds, qubole, rabbitmq, redis, s3, salesforce, samba, segment,
+sendgrid, sentry, sftp, singularity, slack, snowflake, spark, sqlite, ssh, statsd, tableau, tabular,
+telegram, trino, vertica, virtualenv, webhdfs, winrm, yandex, zendesk
   .. END EXTRAS HERE
 
 Provider packages
diff --git a/INSTALL b/INSTALL
index 35b39b0b89..164b9622a3 100644
--- a/INSTALL
+++ b/INSTALL
@@ -97,15 +97,15 @@ The list of available extras:
 airbyte, alibaba, all, all_dbs, amazon, apache.atlas, apache.beam, apache.cassandra, apache.drill,
 apache.druid, apache.hdfs, apache.hive, apache.kylin, apache.livy, apache.pig, apache.pinot,
 apache.spark, apache.sqoop, apache.webhdfs, arangodb, asana, async, atlas, aws, azure, cassandra,
-celery, cgroups, cloudant, cncf.kubernetes, core.sql, crypto, dask, databricks, datadog, dbt.cloud,
-deprecated_api, devel, devel_all, devel_ci, devel_hadoop, dingding, discord, doc, docker, druid,
-elasticsearch, exasol, facebook, ftp, gcp, gcp_api, github, github_enterprise, google, google_auth,
-grpc, hashicorp, hdfs, hive, http, imap, influxdb, jdbc, jenkins, jira, kerberos, kubernetes, ldap,
-leveldb, microsoft.azure, microsoft.mssql, microsoft.psrp, microsoft.winrm, mongo, mssql, mysql,
-neo4j, odbc, openfaas, opsgenie, oracle, pagerduty, pandas, papermill, password, pinot, plexus,
-postgres, presto, qds, qubole, rabbitmq, redis, s3, salesforce, samba, segment, sendgrid, sentry,
-sftp, singularity, slack, snowflake, spark, sqlite, ssh, statsd, tableau, tabular, telegram, trino,
-vertica, virtualenv, webhdfs, winrm, yandex, zendesk
+celery, cgroups, cloudant, cncf.kubernetes, common.sql, crypto, dask, databricks, datadog,
+dbt.cloud, deprecated_api, devel, devel_all, devel_ci, devel_hadoop, dingding, discord, doc, docker,
+druid, elasticsearch, exasol, facebook, ftp, gcp, gcp_api, github, github_enterprise, google,
+google_auth, grpc, hashicorp, hdfs, hive, http, imap, influxdb, jdbc, jenkins, jira, kerberos,
+kubernetes, ldap, leveldb, microsoft.azure, microsoft.mssql, microsoft.psrp, microsoft.winrm, mongo,
+mssql, mysql, neo4j, odbc, openfaas, opsgenie, oracle, pagerduty, pandas, papermill, password,
+pinot, plexus, postgres, presto, qds, qubole, rabbitmq, redis, s3, salesforce, samba, segment,
+sendgrid, sentry, sftp, singularity, slack, snowflake, spark, sqlite, ssh, statsd, tableau, tabular,
+telegram, trino, vertica, virtualenv, webhdfs, winrm, yandex, zendesk
 # END EXTRAS HERE
 
 # For installing Airflow in development environments - see CONTRIBUTING.rst
diff --git a/airflow/hooks/dbapi.py b/airflow/hooks/dbapi.py
index d3d3fa5fc8..1dc2908eb9 100644
--- a/airflow/hooks/dbapi.py
+++ b/airflow/hooks/dbapi.py
@@ -15,367 +15,13 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-from contextlib import closing
-from datetime import datetime
-from typing import Any, Optional
+import warnings
 
-from sqlalchemy import create_engine
+from airflow.providers.common.sql.hooks.sql import ConnectorProtocol  # noqa
+from airflow.providers.common.sql.hooks.sql import DbApiHook  # noqa
 
-from airflow.exceptions import AirflowException
-from airflow.hooks.base import BaseHook
-from airflow.typing_compat import Protocol
-
-
-class ConnectorProtocol(Protocol):
-    """A protocol where you can connect to a database."""
-
-    def connect(self, host: str, port: int, username: str, schema: str) -> Any:
-        """
-        Connect to a database.
-
-        :param host: The database host to connect to.
-        :param port: The database port to connect to.
-        :param username: The database username used for the authentication.
-        :param schema: The database schema to connect to.
-        :return: the authorized connection object.
-        """
-
-
-#########################################################################################
-#                                                                                       #
-#  Note! Be extra careful when changing this file. This hook is used as a base for      #
-#  a number of DBApi-related hooks and providers depend on the methods implemented      #
-#  here. Whatever you add here, has to backwards compatible unless                      #
-#  `>=<Airflow version>` is added to providers' requirements using the new feature      #
-#                                                                                       #
-#########################################################################################
-class DbApiHook(BaseHook):
-    """
-    Abstract base class for sql hooks.
-
-    :param schema: Optional DB schema that overrides the schema specified in the connection. Make sure that
-        if you change the schema parameter value in the constructor of the derived Hook, such change
-        should be done before calling the ``DBApiHook.__init__()``.
-    :param log_sql: Whether to log SQL query when it's executed. Defaults to *True*.
-    """
-
-    # Override to provide the connection name.
-    conn_name_attr = None  # type: str
-    # Override to have a default connection id for a particular dbHook
-    default_conn_name = 'default_conn_id'
-    # Override if this db supports autocommit.
-    supports_autocommit = False
-    # Override with the object that exposes the connect method
-    connector = None  # type: Optional[ConnectorProtocol]
-    # Override with db-specific query to check connection
-    _test_connection_sql = "select 1"
-
-    def __init__(self, *args, schema: Optional[str] = None, log_sql: bool = True, **kwargs):
-        super().__init__()
-        if not self.conn_name_attr:
-            raise AirflowException("conn_name_attr is not defined")
-        elif len(args) == 1:
-            setattr(self, self.conn_name_attr, args[0])
-        elif self.conn_name_attr not in kwargs:
-            setattr(self, self.conn_name_attr, self.default_conn_name)
-        else:
-            setattr(self, self.conn_name_attr, kwargs[self.conn_name_attr])
-        # We should not make schema available in deriving hooks for backwards compatibility
-        # If a hook deriving from DBApiHook has a need to access schema, then it should retrieve it
-        # from kwargs and store it on its own. We do not run "pop" here as we want to give the
-        # Hook deriving from the DBApiHook to still have access to the field in it's constructor
-        self.__schema = schema
-        self.log_sql = log_sql
-
-    def get_conn(self):
-        """Returns a connection object"""
-        db = self.get_connection(getattr(self, self.conn_name_attr))
-        return self.connector.connect(host=db.host, port=db.port, username=db.login, schema=db.schema)
-
-    def get_uri(self) -> str:
-        """
-        Extract the URI from the connection.
-
-        :return: the extracted uri.
-        """
-        conn = self.get_connection(getattr(self, self.conn_name_attr))
-        conn.schema = self.__schema or conn.schema
-        return conn.get_uri()
-
-    def get_sqlalchemy_engine(self, engine_kwargs=None):
-        """
-        Get an sqlalchemy_engine object.
-
-        :param engine_kwargs: Kwargs used in :func:`~sqlalchemy.create_engine`.
-        :return: the created engine.
-        """
-        if engine_kwargs is None:
-            engine_kwargs = {}
-        return create_engine(self.get_uri(), **engine_kwargs)
-
-    def get_pandas_df(self, sql, parameters=None, **kwargs):
-        """
-        Executes the sql and returns a pandas dataframe
-
-        :param sql: the sql statement to be executed (str) or a list of
-            sql statements to execute
-        :param parameters: The parameters to render the SQL query with.
-        :param kwargs: (optional) passed into pandas.io.sql.read_sql method
-        """
-        try:
-            from pandas.io import sql as psql
-        except ImportError:
-            raise Exception("pandas library not installed, run: pip install 'apache-airflow[pandas]'.")
-
-        with closing(self.get_conn()) as conn:
-            return psql.read_sql(sql, con=conn, params=parameters, **kwargs)
-
-    def get_pandas_df_by_chunks(self, sql, parameters=None, *, chunksize, **kwargs):
-        """
-        Executes the sql and returns a generator
-
-        :param sql: the sql statement to be executed (str) or a list of
-            sql statements to execute
-        :param parameters: The parameters to render the SQL query with
-        :param chunksize: number of rows to include in  each chunk
-        :param kwargs: (optional) passed into pandas.io.sql.read_sql method
-        """
-        try:
-            from pandas.io import sql as psql
-        except ImportError:
-            raise Exception("pandas library not installed, run: pip install 'apache-airflow[pandas]'.")
-
-        with closing(self.get_conn()) as conn:
-            yield from psql.read_sql(sql, con=conn, params=parameters, chunksize=chunksize, **kwargs)
-
-    def get_records(self, sql, parameters=None):
-        """
-        Executes the sql and returns a set of records.
-
-        :param sql: the sql statement to be executed (str) or a list of
-            sql statements to execute
-        :param parameters: The parameters to render the SQL query with.
-        """
-        with closing(self.get_conn()) as conn:
-            with closing(conn.cursor()) as cur:
-                if parameters is not None:
-                    cur.execute(sql, parameters)
-                else:
-                    cur.execute(sql)
-                return cur.fetchall()
-
-    def get_first(self, sql, parameters=None):
-        """
-        Executes the sql and returns the first resulting row.
-
-        :param sql: the sql statement to be executed (str) or a list of
-            sql statements to execute
-        :param parameters: The parameters to render the SQL query with.
-        """
-        with closing(self.get_conn()) as conn:
-            with closing(conn.cursor()) as cur:
-                if parameters is not None:
-                    cur.execute(sql, parameters)
-                else:
-                    cur.execute(sql)
-                return cur.fetchone()
-
-    def run(self, sql, autocommit=False, parameters=None, handler=None):
-        """
-        Runs a command or a list of commands. Pass a list of sql
-        statements to the sql parameter to get them to execute
-        sequentially
-
-        :param sql: the sql statement to be executed (str) or a list of
-            sql statements to execute
-        :param autocommit: What to set the connection's autocommit setting to
-            before executing the query.
-        :param parameters: The parameters to render the SQL query with.
-        :param handler: The result handler which is called with the result of each statement.
-        :return: query results if handler was provided.
-        """
-        scalar = isinstance(sql, str)
-        if scalar:
-            sql = [sql]
-
-        if sql:
-            self.log.debug("Executing %d statements", len(sql))
-        else:
-            raise ValueError("List of SQL statements is empty")
-
-        with closing(self.get_conn()) as conn:
-            if self.supports_autocommit:
-                self.set_autocommit(conn, autocommit)
-
-            with closing(conn.cursor()) as cur:
-                results = []
-                for sql_statement in sql:
-                    self._run_command(cur, sql_statement, parameters)
-                    if handler is not None:
-                        result = handler(cur)
-                        results.append(result)
-
-            # If autocommit was set to False for db that supports autocommit,
-            # or if db does not supports autocommit, we do a manual commit.
-            if not self.get_autocommit(conn):
-                conn.commit()
-
-        if handler is None:
-            return None
-
-        if scalar:
-            return results[0]
-
-        return results
-
-    def _run_command(self, cur, sql_statement, parameters):
-        """Runs a statement using an already open cursor."""
-        if self.log_sql:
-            self.log.info("Running statement: %s, parameters: %s", sql_statement, parameters)
-
-        if parameters:
-            cur.execute(sql_statement, parameters)
-        else:
-            cur.execute(sql_statement)
-
-        # According to PEP 249, this is -1 when query result is not applicable.
-        if cur.rowcount >= 0:
-            self.log.info("Rows affected: %s", cur.rowcount)
-
-    def set_autocommit(self, conn, autocommit):
-        """Sets the autocommit flag on the connection"""
-        if not self.supports_autocommit and autocommit:
-            self.log.warning(
-                "%s connection doesn't support autocommit but autocommit activated.",
-                getattr(self, self.conn_name_attr),
-            )
-        conn.autocommit = autocommit
-
-    def get_autocommit(self, conn):
-        """
-        Get autocommit setting for the provided connection.
-        Return True if conn.autocommit is set to True.
-        Return False if conn.autocommit is not set or set to False or conn
-        does not support autocommit.
-
-        :param conn: Connection to get autocommit setting from.
-        :return: connection autocommit setting.
-        :rtype: bool
-        """
-        return getattr(conn, 'autocommit', False) and self.supports_autocommit
-
-    def get_cursor(self):
-        """Returns a cursor"""
-        return self.get_conn().cursor()
-
-    @staticmethod
-    def _generate_insert_sql(table, values, target_fields, replace, **kwargs):
-        """
-        Static helper method that generates the INSERT SQL statement.
-        The REPLACE variant is specific to MySQL syntax.
-
-        :param table: Name of the target table
-        :param values: The row to insert into the table
-        :param target_fields: The names of the columns to fill in the table
-        :param replace: Whether to replace instead of insert
-        :return: The generated INSERT or REPLACE SQL statement
-        :rtype: str
-        """
-        placeholders = [
-            "%s",
-        ] * len(values)
-
-        if target_fields:
-            target_fields = ", ".join(target_fields)
-            target_fields = f"({target_fields})"
-        else:
-            target_fields = ''
-
-        if not replace:
-            sql = "INSERT INTO "
-        else:
-            sql = "REPLACE INTO "
-        sql += f"{table} {target_fields} VALUES ({','.join(placeholders)})"
-        return sql
-
-    def insert_rows(self, table, rows, target_fields=None, commit_every=1000, replace=False, **kwargs):
-        """
-        A generic way to insert a set of tuples into a table,
-        a new transaction is created every commit_every rows
-
-        :param table: Name of the target table
-        :param rows: The rows to insert into the table
-        :param target_fields: The names of the columns to fill in the table
-        :param commit_every: The maximum number of rows to insert in one
-            transaction. Set to 0 to insert all rows in one transaction.
-        :param replace: Whether to replace instead of insert
-        """
-        i = 0
-        with closing(self.get_conn()) as conn:
-            if self.supports_autocommit:
-                self.set_autocommit(conn, False)
-
-            conn.commit()
-
-            with closing(conn.cursor()) as cur:
-                for i, row in enumerate(rows, 1):
-                    lst = []
-                    for cell in row:
-                        lst.append(self._serialize_cell(cell, conn))
-                    values = tuple(lst)
-                    sql = self._generate_insert_sql(table, values, target_fields, replace, **kwargs)
-                    self.log.debug("Generated sql: %s", sql)
-                    cur.execute(sql, values)
-                    if commit_every and i % commit_every == 0:
-                        conn.commit()
-                        self.log.info("Loaded %s rows into %s so far", i, table)
-
-            conn.commit()
-        self.log.info("Done loading. Loaded a total of %s rows", i)
-
-    @staticmethod
-    def _serialize_cell(cell, conn=None):
-        """
-        Returns the SQL literal of the cell as a string.
-
-        :param cell: The cell to insert into the table
-        :param conn: The database connection
-        :return: The serialized cell
-        :rtype: str
-        """
-        if cell is None:
-            return None
-        if isinstance(cell, datetime):
-            return cell.isoformat()
-        return str(cell)
-
-    def bulk_dump(self, table, tmp_file):
-        """
-        Dumps a database table into a tab-delimited file
-
-        :param table: The name of the source table
-        :param tmp_file: The path of the target file
-        """
-        raise NotImplementedError()
-
-    def bulk_load(self, table, tmp_file):
-        """
-        Loads a tab-delimited file into a database table
-
-        :param table: The name of the target table
-        :param tmp_file: The path of the file to load into the table
-        """
-        raise NotImplementedError()
-
-    def test_connection(self):
-        """Tests the connection using db-specific query"""
-        status, message = False, ''
-        try:
-            if self.get_first(self._test_connection_sql):
-                status = True
-                message = 'Connection successfully tested'
-        except Exception as e:
-            status = False
-            message = str(e)
-
-        return status, message
+warnings.warn(
+    "This module is deprecated. Please use `airflow.providers.common.sql.hooks.sql`.",
+    DeprecationWarning,
+    stacklevel=2,
+)
diff --git a/airflow/hooks/dbapi_hook.py b/airflow/hooks/dbapi_hook.py
index 4a441b0f50..6445db7881 100644
--- a/airflow/hooks/dbapi_hook.py
+++ b/airflow/hooks/dbapi_hook.py
@@ -19,8 +19,10 @@
 
 import warnings
 
-from airflow.hooks.dbapi import DbApiHook  # noqa
+from airflow.providers.common.sql.hooks.sql import DbApiHook  # noqa
 
 warnings.warn(
-    "This module is deprecated. Please use `airflow.hooks.dbapi`.", DeprecationWarning, stacklevel=2
+    "This module is deprecated. Please use `airflow.providers.common.sql.hooks.sql`.",
+    DeprecationWarning,
+    stacklevel=2,
 )
diff --git a/airflow/operators/sql.py b/airflow/operators/sql.py
index efa5d0d81a..cb8b664875 100644
--- a/airflow/operators/sql.py
+++ b/airflow/operators/sql.py
@@ -20,8 +20,8 @@ from typing import Any, Dict, Iterable, List, Mapping, Optional, Sequence, Suppo
 from airflow.compat.functools import cached_property
 from airflow.exceptions import AirflowException
 from airflow.hooks.base import BaseHook
-from airflow.hooks.dbapi import DbApiHook
 from airflow.models import BaseOperator, SkipMixin
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 from airflow.utils.context import Context
 
 
diff --git a/airflow/providers/amazon/aws/hooks/redshift_sql.py b/airflow/providers/amazon/aws/hooks/redshift_sql.py
index 692b600b38..304e5cb898 100644
--- a/airflow/providers/amazon/aws/hooks/redshift_sql.py
+++ b/airflow/providers/amazon/aws/hooks/redshift_sql.py
@@ -23,7 +23,7 @@ from sqlalchemy import create_engine
 from sqlalchemy.engine.url import URL
 
 from airflow.compat.functools import cached_property
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 
 class RedshiftSQLHook(DbApiHook):
diff --git a/airflow/providers/amazon/aws/transfers/sql_to_s3.py b/airflow/providers/amazon/aws/transfers/sql_to_s3.py
index f399c27141..d0a15e1609 100644
--- a/airflow/providers/amazon/aws/transfers/sql_to_s3.py
+++ b/airflow/providers/amazon/aws/transfers/sql_to_s3.py
@@ -27,9 +27,9 @@ from typing_extensions import Literal
 
 from airflow.exceptions import AirflowException
 from airflow.hooks.base import BaseHook
-from airflow.hooks.dbapi import DbApiHook
 from airflow.models import BaseOperator
 from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 if TYPE_CHECKING:
     from airflow.utils.context import Context
diff --git a/airflow/providers/amazon/provider.yaml b/airflow/providers/amazon/provider.yaml
index ee7f246298..2311e37905 100644
--- a/airflow/providers/amazon/provider.yaml
+++ b/airflow/providers/amazon/provider.yaml
@@ -43,6 +43,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - boto3>=1.15.0
   # watchtower 3 has been released end Jan and introduced breaking change across the board that might
   # change logging behaviour:
diff --git a/airflow/providers/apache/drill/hooks/drill.py b/airflow/providers/apache/drill/hooks/drill.py
index 5baf1f9ecb..218b31bf18 100644
--- a/airflow/providers/apache/drill/hooks/drill.py
+++ b/airflow/providers/apache/drill/hooks/drill.py
@@ -21,7 +21,7 @@ from typing import Any, Iterable, Optional, Tuple
 from sqlalchemy import create_engine
 from sqlalchemy.engine import Connection
 
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 
 class DrillHook(DbApiHook):
diff --git a/airflow/providers/apache/drill/provider.yaml b/airflow/providers/apache/drill/provider.yaml
index dcdf128e50..127455184e 100644
--- a/airflow/providers/apache/drill/provider.yaml
+++ b/airflow/providers/apache/drill/provider.yaml
@@ -31,6 +31,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - sqlalchemy-drill>=1.1.0
   - sqlparse>=0.4.1
 
diff --git a/airflow/providers/apache/druid/hooks/druid.py b/airflow/providers/apache/druid/hooks/druid.py
index a10519eea4..2415bc4bc1 100644
--- a/airflow/providers/apache/druid/hooks/druid.py
+++ b/airflow/providers/apache/druid/hooks/druid.py
@@ -24,7 +24,7 @@ from pydruid.db import connect
 
 from airflow.exceptions import AirflowException
 from airflow.hooks.base import BaseHook
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 
 class DruidHook(BaseHook):
diff --git a/airflow/providers/apache/druid/provider.yaml b/airflow/providers/apache/druid/provider.yaml
index a1af570a15..0139af4292 100644
--- a/airflow/providers/apache/druid/provider.yaml
+++ b/airflow/providers/apache/druid/provider.yaml
@@ -38,6 +38,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - pydruid>=0.4.1
 
 integrations:
diff --git a/airflow/providers/apache/hive/hooks/hive.py b/airflow/providers/apache/hive/hooks/hive.py
index 559c9727a8..63e4652f20 100644
--- a/airflow/providers/apache/hive/hooks/hive.py
+++ b/airflow/providers/apache/hive/hooks/hive.py
@@ -32,7 +32,7 @@ import unicodecsv as csv
 from airflow.configuration import conf
 from airflow.exceptions import AirflowException
 from airflow.hooks.base import BaseHook
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 from airflow.security import utils
 from airflow.utils.helpers import as_flattened_list
 from airflow.utils.operator_helpers import AIRFLOW_VAR_NAME_FORMAT_MAPPING
diff --git a/airflow/providers/apache/hive/provider.yaml b/airflow/providers/apache/hive/provider.yaml
index cd9321c4cd..285393bc82 100644
--- a/airflow/providers/apache/hive/provider.yaml
+++ b/airflow/providers/apache/hive/provider.yaml
@@ -40,6 +40,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - hmsclient>=0.1.0
   - pandas>=0.17.1
   - pyhive[hive]>=0.6.0
diff --git a/airflow/providers/apache/hive/sensors/metastore_partition.py b/airflow/providers/apache/hive/sensors/metastore_partition.py
index ea6c1525a1..f4295f0d00 100644
--- a/airflow/providers/apache/hive/sensors/metastore_partition.py
+++ b/airflow/providers/apache/hive/sensors/metastore_partition.py
@@ -17,7 +17,7 @@
 # under the License.
 from typing import TYPE_CHECKING, Any, Sequence
 
-from airflow.sensors.sql import SqlSensor
+from airflow.providers.common.sql.sensors.sql import SqlSensor
 
 if TYPE_CHECKING:
     from airflow.utils.context import Context
diff --git a/airflow/providers/apache/pinot/hooks/pinot.py b/airflow/providers/apache/pinot/hooks/pinot.py
index 4943b37adc..fa31b9f33d 100644
--- a/airflow/providers/apache/pinot/hooks/pinot.py
+++ b/airflow/providers/apache/pinot/hooks/pinot.py
@@ -24,8 +24,8 @@ from pinotdb import connect
 
 from airflow.exceptions import AirflowException
 from airflow.hooks.base import BaseHook
-from airflow.hooks.dbapi import DbApiHook
 from airflow.models import Connection
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 
 class PinotAdminHook(BaseHook):
diff --git a/airflow/providers/apache/pinot/provider.yaml b/airflow/providers/apache/pinot/provider.yaml
index 8c2708c2fb..d2f1f0e850 100644
--- a/airflow/providers/apache/pinot/provider.yaml
+++ b/airflow/providers/apache/pinot/provider.yaml
@@ -33,6 +33,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   # 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
diff --git a/airflow/providers/core/__init__.py b/airflow/providers/common/__init__.py
similarity index 100%
copy from airflow/providers/core/__init__.py
copy to airflow/providers/common/__init__.py
diff --git a/airflow/providers/core/sql/CHANGELOG.rst b/airflow/providers/common/sql/CHANGELOG.rst
similarity index 86%
rename from airflow/providers/core/sql/CHANGELOG.rst
rename to airflow/providers/common/sql/CHANGELOG.rst
index a5b4657ed3..e8cd07d33b 100644
--- a/airflow/providers/core/sql/CHANGELOG.rst
+++ b/airflow/providers/common/sql/CHANGELOG.rst
@@ -23,4 +23,5 @@ Changelog
 .....
 
 Initial version of the provider.
-Adds SQLColumnCheckOperator and SQLTableCheckOperator.
+Adds ``SQLColumnCheckOperator`` and ``SQLTableCheckOperator``.
+Moves ``DBApiHook``, ``SQLSensor`` and ``ConnectorProtocol`` to the provider.
diff --git a/airflow/providers/core/sql/__init__.py b/airflow/providers/common/sql/__init__.py
similarity index 100%
rename from airflow/providers/core/sql/__init__.py
rename to airflow/providers/common/sql/__init__.py
diff --git a/airflow/providers/core/sql/example_dags/__init__.py b/airflow/providers/common/sql/hooks/__init__.py
similarity index 100%
rename from airflow/providers/core/sql/example_dags/__init__.py
rename to airflow/providers/common/sql/hooks/__init__.py
diff --git a/airflow/hooks/dbapi.py b/airflow/providers/common/sql/hooks/sql.py
similarity index 91%
copy from airflow/hooks/dbapi.py
copy to airflow/providers/common/sql/hooks/sql.py
index d3d3fa5fc8..efd4a9dcfe 100644
--- a/airflow/hooks/dbapi.py
+++ b/airflow/providers/common/sql/hooks/sql.py
@@ -1,4 +1,3 @@
-#
 # 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
@@ -15,15 +14,39 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+import warnings
 from contextlib import closing
 from datetime import datetime
 from typing import Any, Optional
 
 from sqlalchemy import create_engine
+from typing_extensions import Protocol
 
-from airflow.exceptions import AirflowException
+from airflow import AirflowException
 from airflow.hooks.base import BaseHook
-from airflow.typing_compat import Protocol
+from airflow.providers_manager import ProvidersManager
+from airflow.utils.module_loading import import_string
+
+
+def _backported_get_hook(connection, *, hook_params=None):
+    """Return hook based on conn_type
+    For supporting Airflow versions < 2.3, we backport "get_hook()" method. This should be removed
+    when "apache-airflow-providers-slack" will depend on Airflow >= 2.3.
+    """
+    hook = ProvidersManager().hooks.get(connection.conn_type, None)
+
+    if hook is None:
+        raise AirflowException(f'Unknown hook type "{connection.conn_type}"')
+    try:
+        hook_class = import_string(hook.hook_class_name)
+    except ImportError:
+        warnings.warn(
+            f"Could not import {hook.hook_class_name} when discovering {hook.hook_name} {hook.package_name}",
+        )
+        raise
+    if hook_params is None:
+        hook_params = {}
+    return hook_class(**{hook.connection_id_attribute_name: connection.conn_id}, **hook_params)
 
 
 class ConnectorProtocol(Protocol):
@@ -41,14 +64,6 @@ class ConnectorProtocol(Protocol):
         """
 
 
-#########################################################################################
-#                                                                                       #
-#  Note! Be extra careful when changing this file. This hook is used as a base for      #
-#  a number of DBApi-related hooks and providers depend on the methods implemented      #
-#  here. Whatever you add here, has to backwards compatible unless                      #
-#  `>=<Airflow version>` is added to providers' requirements using the new feature      #
-#                                                                                       #
-#########################################################################################
 class DbApiHook(BaseHook):
     """
     Abstract base class for sql hooks.
@@ -125,7 +140,10 @@ class DbApiHook(BaseHook):
         try:
             from pandas.io import sql as psql
         except ImportError:
-            raise Exception("pandas library not installed, run: pip install 'apache-airflow[pandas]'.")
+            raise Exception(
+                "pandas library not installed, run: pip install "
+                "'apache-airflow-providers-common-sql[pandas]'."
+            )
 
         with closing(self.get_conn()) as conn:
             return psql.read_sql(sql, con=conn, params=parameters, **kwargs)
@@ -143,7 +161,10 @@ class DbApiHook(BaseHook):
         try:
             from pandas.io import sql as psql
         except ImportError:
-            raise Exception("pandas library not installed, run: pip install 'apache-airflow[pandas]'.")
+            raise Exception(
+                "pandas library not installed, run: pip install "
+                "'apache-airflow-providers-common-sql[pandas]'."
+            )
 
         with closing(self.get_conn()) as conn:
             yield from psql.read_sql(sql, con=conn, params=parameters, chunksize=chunksize, **kwargs)
diff --git a/airflow/providers/core/sql/operators/__init__.py b/airflow/providers/common/sql/operators/__init__.py
similarity index 100%
rename from airflow/providers/core/sql/operators/__init__.py
rename to airflow/providers/common/sql/operators/__init__.py
diff --git a/airflow/providers/core/sql/operators/sql.py b/airflow/providers/common/sql/operators/sql.py
similarity index 99%
rename from airflow/providers/core/sql/operators/sql.py
rename to airflow/providers/common/sql/operators/sql.py
index 1d9945f403..63b6457c75 100644
--- a/airflow/providers/core/sql/operators/sql.py
+++ b/airflow/providers/common/sql/operators/sql.py
@@ -121,7 +121,7 @@ class SQLColumnCheckOperator(BaseSQLOperator):
             if not records:
                 raise AirflowException(f"The following query returned zero rows: {self.sql}")
 
-            self.log.info(f"Record: {records}")
+            self.log.info("Record: %s", records)
 
             for idx, result in enumerate(records):
                 tolerance = self.column_mapping[column][checks[idx]].get("tolerance")
@@ -298,7 +298,7 @@ class SQLTableCheckOperator(BaseSQLOperator):
         if not records:
             raise AirflowException(f"The following query returned zero rows: {self.sql}")
 
-        self.log.info(f"Record: {records}")
+        self.log.info("Record: %s", records)
 
         for check in self.checks.keys():
             for result in records:
diff --git a/airflow/providers/core/sql/provider.yaml b/airflow/providers/common/sql/provider.yaml
similarity index 61%
rename from airflow/providers/core/sql/provider.yaml
rename to airflow/providers/common/sql/provider.yaml
index f951984267..a277f327cc 100644
--- a/airflow/providers/core/sql/provider.yaml
+++ b/airflow/providers/common/sql/provider.yaml
@@ -16,26 +16,40 @@
 # under the License.
 
 ---
-package-name: apache-airflow-providers-core-sql
-name: Core SQL
+package-name: apache-airflow-providers-common-sql
+name: Common SQL
 description: |
-    `Core SQL Provider <https://en.wikipedia.org/wiki/SQL>`__
+    `Common SQL Provider <https://en.wikipedia.org/wiki/SQL>`__
 
 versions:
   - 1.0.0
 
-dependencies:
-  - apache-airflow>=2.2.0
+dependencies: []
+
+additional-extras:
+  - name: pandas
+    dependencies:
+      - pandas>=0.17.1
 
 integrations:
-  - integration-name: Core SQL
+  - integration-name: Common SQL
     external-doc-url: https://en.wikipedia.org/wiki/SQL
     how-to-guide:
-      - /docs/apache-airflow-providers-core-sql/operators.rst
+      - /docs/apache-airflow-providers-common-sql/operators.rst
     logo: /integration-logos/core/sql/sql.png
     tags: [software]
 
 operators:
-  - integration-name: Core SQL
+  - integration-name: Common SQL
+    python-modules:
+      - airflow.providers.common.sql.operators.sql
+
+hooks:
+  - integration-name: Common SQL
+    python-modules:
+      - airflow.providers.common.sql.hooks.sql
+
+sensors:
+  - integration-name: Common SQL
     python-modules:
-      - airflow.providers.core.sql.operators.sql
+      - airflow.providers.common.sql.sensors.sql
diff --git a/tests/providers/core/__init__.py b/airflow/providers/common/sql/sensors/__init__.py
similarity index 100%
rename from tests/providers/core/__init__.py
rename to airflow/providers/common/sql/sensors/__init__.py
diff --git a/airflow/sensors/sql.py b/airflow/providers/common/sql/sensors/sql.py
similarity index 85%
copy from airflow/sensors/sql.py
copy to airflow/providers/common/sql/sensors/sql.py
index a35d7566ce..c9f66568e6 100644
--- a/airflow/sensors/sql.py
+++ b/airflow/providers/common/sql/sensors/sql.py
@@ -1,4 +1,3 @@
-#
 # 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
@@ -16,13 +15,15 @@
 # specific language governing permissions and limitations
 # under the License.
 
-from typing import Sequence
+from typing import Any, Sequence
+
+from packaging.version import Version
 
-from airflow.exceptions import AirflowException
+from airflow import AirflowException
 from airflow.hooks.base import BaseHook
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook, _backported_get_hook
 from airflow.sensors.base import BaseSensorOperator
-from airflow.utils.context import Context
+from airflow.version import version
 
 
 class SqlSensor(BaseSensorOperator):
@@ -78,7 +79,13 @@ class SqlSensor(BaseSensorOperator):
 
     def _get_hook(self):
         conn = BaseHook.get_connection(self.conn_id)
-        hook = conn.get_hook(hook_params=self.hook_params)
+        if Version(version) >= Version('2.3'):
+            # "hook_params" were introduced to into "get_hook()" only in Airflow 2.3.
+            hook = conn.get_hook(hook_params=self.hook_params)  # ignore airflow compat check
+        else:
+            # For supporting Airflow versions < 2.3, we backport "get_hook()" method. This should be removed
+            # when "apache-airflow-providers-common-sql" will depend on Airflow >= 2.3.
+            hook = _backported_get_hook(conn, hook_params=self.hook_params)
         if not isinstance(hook, DbApiHook):
             raise AirflowException(
                 f'The connection type is not supported by {self.__class__.__name__}. '
@@ -86,7 +93,7 @@ class SqlSensor(BaseSensorOperator):
             )
         return hook
 
-    def poke(self, context: Context):
+    def poke(self, context: Any):
         hook = self._get_hook()
 
         self.log.info('Poking: %s (with parameters %s)', self.sql, self.parameters)
diff --git a/airflow/providers/databricks/hooks/databricks_sql.py b/airflow/providers/databricks/hooks/databricks_sql.py
index 9d86b4dbe3..6c5800170d 100644
--- a/airflow/providers/databricks/hooks/databricks_sql.py
+++ b/airflow/providers/databricks/hooks/databricks_sql.py
@@ -25,7 +25,7 @@ from databricks.sql.client import Connection  # type: ignore[attr-defined]
 
 from airflow import __version__
 from airflow.exceptions import AirflowException
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 from airflow.providers.databricks.hooks.databricks_base import BaseDatabricksHook
 
 LIST_SQL_ENDPOINTS_ENDPOINT = ('GET', 'api/2.0/sql/endpoints')
diff --git a/airflow/providers/databricks/provider.yaml b/airflow/providers/databricks/provider.yaml
index d20878bef5..b0d4d44bc4 100644
--- a/airflow/providers/databricks/provider.yaml
+++ b/airflow/providers/databricks/provider.yaml
@@ -38,6 +38,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - requests>=2.27,<3
   - databricks-sql-connector>=2.0.0, <3.0.0
   - aiohttp>=3.6.3, <4
diff --git a/airflow/providers/elasticsearch/hooks/elasticsearch.py b/airflow/providers/elasticsearch/hooks/elasticsearch.py
index b48511670f..493ca7f082 100644
--- a/airflow/providers/elasticsearch/hooks/elasticsearch.py
+++ b/airflow/providers/elasticsearch/hooks/elasticsearch.py
@@ -20,8 +20,8 @@ from typing import Optional
 
 from es.elastic.api import Connection as ESConnection, connect
 
-from airflow.hooks.dbapi import DbApiHook
 from airflow.models.connection import Connection as AirflowConnection
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 
 class ElasticsearchHook(DbApiHook):
diff --git a/airflow/providers/elasticsearch/provider.yaml b/airflow/providers/elasticsearch/provider.yaml
index e9f4a2ddca..25ef669963 100644
--- a/airflow/providers/elasticsearch/provider.yaml
+++ b/airflow/providers/elasticsearch/provider.yaml
@@ -40,6 +40,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - elasticsearch>7
   - elasticsearch-dbapi
   - elasticsearch-dsl>=5.0.0
diff --git a/airflow/providers/exasol/hooks/exasol.py b/airflow/providers/exasol/hooks/exasol.py
index 2233ce1e2c..784c57cde0 100644
--- a/airflow/providers/exasol/hooks/exasol.py
+++ b/airflow/providers/exasol/hooks/exasol.py
@@ -23,7 +23,7 @@ import pandas as pd
 import pyexasol
 from pyexasol import ExaConnection
 
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 
 class ExasolHook(DbApiHook):
diff --git a/airflow/providers/exasol/provider.yaml b/airflow/providers/exasol/provider.yaml
index b0594fb653..da8479d959 100644
--- a/airflow/providers/exasol/provider.yaml
+++ b/airflow/providers/exasol/provider.yaml
@@ -35,6 +35,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - pyexasol>=0.5.1
   - pandas>=0.17.1
 
diff --git a/airflow/providers/google/cloud/hooks/bigquery.py b/airflow/providers/google/cloud/hooks/bigquery.py
index 1be077d790..0049143aea 100644
--- a/airflow/providers/google/cloud/hooks/bigquery.py
+++ b/airflow/providers/google/cloud/hooks/bigquery.py
@@ -52,7 +52,7 @@ from pandas_gbq.gbq import GbqConnector  # noqa
 from sqlalchemy import create_engine
 
 from airflow.exceptions import AirflowException
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 from airflow.providers.google.common.consts import CLIENT_INFO
 from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
 from airflow.utils.helpers import convert_camel_to_snake
diff --git a/airflow/providers/google/provider.yaml b/airflow/providers/google/provider.yaml
index a4962fb714..5d6c0b739e 100644
--- a/airflow/providers/google/provider.yaml
+++ b/airflow/providers/google/provider.yaml
@@ -52,6 +52,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   # 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
diff --git a/airflow/providers/jdbc/hooks/jdbc.py b/airflow/providers/jdbc/hooks/jdbc.py
index 734afecb5b..a33a2b31ee 100644
--- a/airflow/providers/jdbc/hooks/jdbc.py
+++ b/airflow/providers/jdbc/hooks/jdbc.py
@@ -20,8 +20,8 @@ from typing import Any, Dict, Optional
 
 import jaydebeapi
 
-from airflow.hooks.dbapi import DbApiHook
 from airflow.models.connection import Connection
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 
 class JdbcHook(DbApiHook):
diff --git a/airflow/providers/jdbc/provider.yaml b/airflow/providers/jdbc/provider.yaml
index f12f371ab8..958a7e65e5 100644
--- a/airflow/providers/jdbc/provider.yaml
+++ b/airflow/providers/jdbc/provider.yaml
@@ -34,6 +34,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - jaydebeapi>=1.1.1
 
 integrations:
diff --git a/airflow/providers/microsoft/mssql/hooks/mssql.py b/airflow/providers/microsoft/mssql/hooks/mssql.py
index 75241f1f29..ba236efbc2 100644
--- a/airflow/providers/microsoft/mssql/hooks/mssql.py
+++ b/airflow/providers/microsoft/mssql/hooks/mssql.py
@@ -20,7 +20,7 @@
 
 import pymssql
 
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 
 class MsSqlHook(DbApiHook):
diff --git a/airflow/providers/microsoft/mssql/operators/mssql.py b/airflow/providers/microsoft/mssql/operators/mssql.py
index 7082c7c52d..5a5738eb68 100644
--- a/airflow/providers/microsoft/mssql/operators/mssql.py
+++ b/airflow/providers/microsoft/mssql/operators/mssql.py
@@ -23,7 +23,7 @@ from airflow.providers.microsoft.mssql.hooks.mssql import MsSqlHook
 from airflow.www import utils as wwwutils
 
 if TYPE_CHECKING:
-    from airflow.hooks.dbapi import DbApiHook
+    from airflow.providers.common.sql.hooks.sql import DbApiHook
     from airflow.utils.context import Context
 
 
diff --git a/airflow/providers/microsoft/mssql/provider.yaml b/airflow/providers/microsoft/mssql/provider.yaml
index b2feb654fd..9f3a35daaf 100644
--- a/airflow/providers/microsoft/mssql/provider.yaml
+++ b/airflow/providers/microsoft/mssql/provider.yaml
@@ -35,6 +35,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - pymssql>=2.1.5; platform_machine != "aarch64"
 
 integrations:
diff --git a/airflow/providers/mysql/hooks/mysql.py b/airflow/providers/mysql/hooks/mysql.py
index 9f0cb58cd4..586cfb5b8c 100644
--- a/airflow/providers/mysql/hooks/mysql.py
+++ b/airflow/providers/mysql/hooks/mysql.py
@@ -20,8 +20,8 @@
 import json
 from typing import TYPE_CHECKING, Dict, Optional, Tuple, Union
 
-from airflow.hooks.dbapi import DbApiHook
 from airflow.models import Connection
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 if TYPE_CHECKING:
     from mysql.connector.abstracts import MySQLConnectionAbstract
diff --git a/airflow/providers/mysql/provider.yaml b/airflow/providers/mysql/provider.yaml
index 21fa45f649..7947686519 100644
--- a/airflow/providers/mysql/provider.yaml
+++ b/airflow/providers/mysql/provider.yaml
@@ -37,6 +37,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - mysql-connector-python>=8.0.11; platform_machine != "aarch64"
   - mysqlclient>=1.3.6; platform_machine != "aarch64"
 
diff --git a/airflow/providers/odbc/hooks/odbc.py b/airflow/providers/odbc/hooks/odbc.py
index 9ce32fa337..d5e288bb3d 100644
--- a/airflow/providers/odbc/hooks/odbc.py
+++ b/airflow/providers/odbc/hooks/odbc.py
@@ -20,7 +20,7 @@ from urllib.parse import quote_plus
 
 import pyodbc
 
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 from airflow.utils.helpers import merge_dicts
 
 
@@ -178,7 +178,10 @@ class OdbcHook(DbApiHook):
         return conn
 
     def get_uri(self) -> str:
-        """URI invoked in :py:meth:`~airflow.hooks.dbapi.DbApiHook.get_sqlalchemy_engine` method"""
+        """
+        URI invoked in :py:meth:`~airflow.providers.common.sql.hooks.sql.DbApiHook.get_sqlalchemy_engine`
+        method.
+        """
         quoted_conn_str = quote_plus(self.odbc_connection_string)
         uri = f"{self.sqlalchemy_scheme}:///?odbc_connect={quoted_conn_str}"
         return uri
diff --git a/airflow/providers/odbc/provider.yaml b/airflow/providers/odbc/provider.yaml
index 12dc46f5eb..18e980038b 100644
--- a/airflow/providers/odbc/provider.yaml
+++ b/airflow/providers/odbc/provider.yaml
@@ -33,6 +33,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - pyodbc
 
 integrations:
diff --git a/airflow/providers/oracle/hooks/oracle.py b/airflow/providers/oracle/hooks/oracle.py
index eb94b435e8..65db1ee7ab 100644
--- a/airflow/providers/oracle/hooks/oracle.py
+++ b/airflow/providers/oracle/hooks/oracle.py
@@ -28,7 +28,7 @@ try:
 except ImportError:
     numpy = None  # type: ignore
 
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 PARAM_TYPES = {bool, float, int, str}
 
diff --git a/airflow/providers/oracle/provider.yaml b/airflow/providers/oracle/provider.yaml
index 513cc34fc0..135e04c43e 100644
--- a/airflow/providers/oracle/provider.yaml
+++ b/airflow/providers/oracle/provider.yaml
@@ -37,6 +37,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - oracledb>=1.0.0
 
 integrations:
diff --git a/airflow/providers/postgres/hooks/postgres.py b/airflow/providers/postgres/hooks/postgres.py
index 884a1c9292..09c07c9b8f 100644
--- a/airflow/providers/postgres/hooks/postgres.py
+++ b/airflow/providers/postgres/hooks/postgres.py
@@ -26,8 +26,8 @@ import psycopg2.extras
 from psycopg2.extensions import connection
 from psycopg2.extras import DictCursor, NamedTupleCursor, RealDictCursor
 
-from airflow.hooks.dbapi import DbApiHook
 from airflow.models.connection import Connection
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 CursorType = Union[DictCursor, RealDictCursor, NamedTupleCursor]
 
diff --git a/airflow/providers/postgres/provider.yaml b/airflow/providers/postgres/provider.yaml
index 9c4c70ad3e..8f81558f47 100644
--- a/airflow/providers/postgres/provider.yaml
+++ b/airflow/providers/postgres/provider.yaml
@@ -38,6 +38,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - psycopg2-binary>=2.7.4
 
 integrations:
diff --git a/airflow/providers/presto/hooks/presto.py b/airflow/providers/presto/hooks/presto.py
index 95ecf86b52..22afc71577 100644
--- a/airflow/providers/presto/hooks/presto.py
+++ b/airflow/providers/presto/hooks/presto.py
@@ -26,8 +26,8 @@ from prestodb.transaction import IsolationLevel
 
 from airflow import AirflowException
 from airflow.configuration import conf
-from airflow.hooks.dbapi import DbApiHook
 from airflow.models import Connection
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 from airflow.utils.operator_helpers import AIRFLOW_VAR_NAME_FORMAT_MAPPING
 
 try:
diff --git a/airflow/providers/presto/provider.yaml b/airflow/providers/presto/provider.yaml
index 26d7db6f8a..dd665140ee 100644
--- a/airflow/providers/presto/provider.yaml
+++ b/airflow/providers/presto/provider.yaml
@@ -36,6 +36,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - presto-python-client>=0.8.2
   - pandas>=0.17.1
 
diff --git a/airflow/providers/qubole/hooks/qubole_check.py b/airflow/providers/qubole/hooks/qubole_check.py
index 5dba31c5cb..59ae4396a3 100644
--- a/airflow/providers/qubole/hooks/qubole_check.py
+++ b/airflow/providers/qubole/hooks/qubole_check.py
@@ -23,7 +23,7 @@ from typing import List, Optional, Union
 from qds_sdk.commands import Command
 
 from airflow.exceptions import AirflowException
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 from airflow.providers.qubole.hooks.qubole import QuboleHook
 
 log = logging.getLogger(__name__)
diff --git a/airflow/providers/qubole/provider.yaml b/airflow/providers/qubole/provider.yaml
index 1826d2299c..a351689313 100644
--- a/airflow/providers/qubole/provider.yaml
+++ b/airflow/providers/qubole/provider.yaml
@@ -35,6 +35,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - qds-sdk>=1.10.4
 
 integrations:
diff --git a/airflow/providers/slack/provider.yaml b/airflow/providers/slack/provider.yaml
index 2a8331d1ac..57ff668b08 100644
--- a/airflow/providers/slack/provider.yaml
+++ b/airflow/providers/slack/provider.yaml
@@ -36,6 +36,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - apache-airflow-providers-http
   - slack_sdk>=3.0.0
 
diff --git a/airflow/providers/slack/transfers/sql_to_slack.py b/airflow/providers/slack/transfers/sql_to_slack.py
index 8b9ff04af3..58c723feca 100644
--- a/airflow/providers/slack/transfers/sql_to_slack.py
+++ b/airflow/providers/slack/transfers/sql_to_slack.py
@@ -14,49 +14,23 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-import warnings
 from typing import TYPE_CHECKING, Iterable, Mapping, Optional, Sequence, Union
 
+from packaging.version import Version
 from pandas import DataFrame
 from tabulate import tabulate
 
 from airflow.exceptions import AirflowException
 from airflow.hooks.base import BaseHook
-from airflow.hooks.dbapi import DbApiHook
 from airflow.models import BaseOperator
+from airflow.providers.common.sql.hooks.sql import DbApiHook, _backported_get_hook
 from airflow.providers.slack.hooks.slack_webhook import SlackWebhookHook
-from airflow.providers_manager import ProvidersManager
-from airflow.utils.module_loading import import_string
 from airflow.version import version
 
 if TYPE_CHECKING:
     from airflow.utils.context import Context
 
 
-def _backported_get_hook(connection, *, hook_params=None):
-    """Return hook based on conn_type
-    For supporting Airflow versions < 2.3, we backport "get_hook()" method. This should be removed
-    when "apache-airflow-providers-slack" will depend on Airflow >= 2.3.
-    """
-    hook = ProvidersManager().hooks.get(connection.conn_type, None)
-
-    if hook is None:
-        raise AirflowException(f'Unknown hook type "{connection.conn_type}"')
-    try:
-        hook_class = import_string(hook.hook_class_name)
-    except ImportError:
-        warnings.warn(
-            "Could not import %s when discovering %s %s",
-            hook.hook_class_name,
-            hook.hook_name,
-            hook.package_name,
-        )
-        raise
-    if hook_params is None:
-        hook_params = {}
-    return hook_class(**{hook.connection_id_attribute_name: connection.conn_id}, **hook_params)
-
-
 class SqlToSlackOperator(BaseOperator):
     """
     Executes an SQL statement in a given SQL connection and sends the results to Slack. The results of the
@@ -126,7 +100,7 @@ class SqlToSlackOperator(BaseOperator):
     def _get_hook(self) -> DbApiHook:
         self.log.debug("Get connection for %s", self.sql_conn_id)
         conn = BaseHook.get_connection(self.sql_conn_id)
-        if version >= '2.3':
+        if Version(version) >= Version('2.3'):
             # "hook_params" were introduced to into "get_hook()" only in Airflow 2.3.
             hook = conn.get_hook(hook_params=self.sql_hook_params)  # ignore airflow compat check
         else:
diff --git a/airflow/providers/snowflake/hooks/snowflake.py b/airflow/providers/snowflake/hooks/snowflake.py
index 29a4b63156..3dee0989ed 100644
--- a/airflow/providers/snowflake/hooks/snowflake.py
+++ b/airflow/providers/snowflake/hooks/snowflake.py
@@ -30,7 +30,7 @@ from snowflake.sqlalchemy import URL
 from sqlalchemy import create_engine
 
 from airflow import AirflowException
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 from airflow.utils.strings import to_boolean
 
 
diff --git a/airflow/providers/snowflake/provider.yaml b/airflow/providers/snowflake/provider.yaml
index c1c035bde4..f097795685 100644
--- a/airflow/providers/snowflake/provider.yaml
+++ b/airflow/providers/snowflake/provider.yaml
@@ -43,6 +43,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - snowflake-connector-python>=2.4.1
   - snowflake-sqlalchemy>=1.1.0
 
diff --git a/airflow/providers/sqlite/hooks/sqlite.py b/airflow/providers/sqlite/hooks/sqlite.py
index 6fe055c149..8559b30401 100644
--- a/airflow/providers/sqlite/hooks/sqlite.py
+++ b/airflow/providers/sqlite/hooks/sqlite.py
@@ -18,7 +18,7 @@
 
 import sqlite3
 
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 
 class SqliteHook(DbApiHook):
diff --git a/airflow/providers/sqlite/provider.yaml b/airflow/providers/sqlite/provider.yaml
index d663a06e3c..d69853378d 100644
--- a/airflow/providers/sqlite/provider.yaml
+++ b/airflow/providers/sqlite/provider.yaml
@@ -33,7 +33,8 @@ versions:
   - 1.0.1
   - 1.0.0
 
-dependencies: []
+dependencies:
+  - apache-airflow-providers-common-sql
 
 integrations:
   - integration-name: SQLite
diff --git a/airflow/providers/trino/hooks/trino.py b/airflow/providers/trino/hooks/trino.py
index cd8fa78c67..9170e19a54 100644
--- a/airflow/providers/trino/hooks/trino.py
+++ b/airflow/providers/trino/hooks/trino.py
@@ -29,8 +29,8 @@ from trino.transaction import IsolationLevel
 
 from airflow import AirflowException
 from airflow.configuration import conf
-from airflow.hooks.dbapi import DbApiHook
 from airflow.models import Connection
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 from airflow.utils.operator_helpers import AIRFLOW_VAR_NAME_FORMAT_MAPPING
 
 try:
diff --git a/airflow/providers/trino/provider.yaml b/airflow/providers/trino/provider.yaml
index f701e6c899..3195e764b0 100644
--- a/airflow/providers/trino/provider.yaml
+++ b/airflow/providers/trino/provider.yaml
@@ -35,6 +35,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - pandas>=0.17.1
   - trino>=0.301.0
 
diff --git a/airflow/providers/vertica/hooks/vertica.py b/airflow/providers/vertica/hooks/vertica.py
index 9530c5965c..5a6e1c125c 100644
--- a/airflow/providers/vertica/hooks/vertica.py
+++ b/airflow/providers/vertica/hooks/vertica.py
@@ -19,7 +19,7 @@
 
 from vertica_python import connect
 
-from airflow.hooks.dbapi import DbApiHook
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 
 class VerticaHook(DbApiHook):
diff --git a/airflow/providers/vertica/provider.yaml b/airflow/providers/vertica/provider.yaml
index 0ebe1989ac..62a8921780 100644
--- a/airflow/providers/vertica/provider.yaml
+++ b/airflow/providers/vertica/provider.yaml
@@ -34,6 +34,7 @@ versions:
 
 dependencies:
   - apache-airflow>=2.2.0
+  - apache-airflow-providers-common-sql
   - vertica-python>=0.5.1
 
 integrations:
diff --git a/airflow/sensors/sql.py b/airflow/sensors/sql.py
index a35d7566ce..c52fe69143 100644
--- a/airflow/sensors/sql.py
+++ b/airflow/sensors/sql.py
@@ -15,97 +15,12 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+import warnings
 
-from typing import Sequence
+from airflow.providers.common.sql.sensors.sql import SqlSensor  # noqa
 
-from airflow.exceptions import AirflowException
-from airflow.hooks.base import BaseHook
-from airflow.hooks.dbapi import DbApiHook
-from airflow.sensors.base import BaseSensorOperator
-from airflow.utils.context import Context
-
-
-class SqlSensor(BaseSensorOperator):
-    """
-    Runs a sql statement repeatedly until a criteria is met. It will keep trying until
-    success or failure criteria are met, or if the first cell is not in (0, '0', '', None).
-    Optional success and failure callables are called with the first cell returned as the argument.
-    If success callable is defined the sensor will keep retrying until the criteria is met.
-    If failure callable is defined and the criteria is met the sensor will raise AirflowException.
-    Failure criteria is evaluated before success criteria. A fail_on_empty boolean can also
-    be passed to the sensor in which case it will fail if no rows have been returned
-
-    :param conn_id: The connection to run the sensor against
-    :param sql: The sql to run. To pass, it needs to return at least one cell
-        that contains a non-zero / empty string value.
-    :param parameters: The parameters to render the SQL query with (optional).
-    :param success: Success criteria for the sensor is a Callable that takes first_cell
-        as the only argument, and returns a boolean (optional).
-    :param failure: Failure criteria for the sensor is a Callable that takes first_cell
-        as the only argument and return a boolean (optional).
-    :param fail_on_empty: Explicitly fail on no rows returned.
-    :param hook_params: Extra config params to be passed to the underlying hook.
-            Should match the desired hook constructor params.
-    """
-
-    template_fields: Sequence[str] = ('sql',)
-    template_ext: Sequence[str] = (
-        '.hql',
-        '.sql',
-    )
-    ui_color = '#7c7287'
-
-    def __init__(
-        self,
-        *,
-        conn_id,
-        sql,
-        parameters=None,
-        success=None,
-        failure=None,
-        fail_on_empty=False,
-        hook_params=None,
-        **kwargs,
-    ):
-        self.conn_id = conn_id
-        self.sql = sql
-        self.parameters = parameters
-        self.success = success
-        self.failure = failure
-        self.fail_on_empty = fail_on_empty
-        self.hook_params = hook_params
-        super().__init__(**kwargs)
-
-    def _get_hook(self):
-        conn = BaseHook.get_connection(self.conn_id)
-        hook = conn.get_hook(hook_params=self.hook_params)
-        if not isinstance(hook, DbApiHook):
-            raise AirflowException(
-                f'The connection type is not supported by {self.__class__.__name__}. '
-                f'The associated hook should be a subclass of `DbApiHook`. Got {hook.__class__.__name__}'
-            )
-        return hook
-
-    def poke(self, context: Context):
-        hook = self._get_hook()
-
-        self.log.info('Poking: %s (with parameters %s)', self.sql, self.parameters)
-        records = hook.get_records(self.sql, self.parameters)
-        if not records:
-            if self.fail_on_empty:
-                raise AirflowException("No rows returned, raising as per fail_on_empty flag")
-            else:
-                return False
-        first_cell = records[0][0]
-        if self.failure is not None:
-            if callable(self.failure):
-                if self.failure(first_cell):
-                    raise AirflowException(f"Failure criteria met. self.failure({first_cell}) returned True")
-            else:
-                raise AirflowException(f"self.failure is present, but not callable -> {self.failure}")
-        if self.success is not None:
-            if callable(self.success):
-                return self.success(first_cell)
-            else:
-                raise AirflowException(f"self.success is present, but not callable -> {self.success}")
-        return bool(first_cell)
+warnings.warn(
+    "This module is deprecated. Please use `airflow.providers.common.sql.sensors.sql`.",
+    DeprecationWarning,
+    stacklevel=2,
+)
diff --git a/airflow/sensors/sql_sensor.py b/airflow/sensors/sql_sensor.py
index 8a077db534..fafc5335a2 100644
--- a/airflow/sensors/sql_sensor.py
+++ b/airflow/sensors/sql_sensor.py
@@ -19,8 +19,10 @@
 
 import warnings
 
-from airflow.sensors.sql import SqlSensor  # noqa
+from airflow.providers.common.sql.sensors.sql import SqlSensor  # noqa
 
 warnings.warn(
-    "This module is deprecated. Please use `airflow.sensors.sql`.", DeprecationWarning, stacklevel=2
+    "This module is deprecated. Please use `airflow.providers.common.sql.sensors.sql`.",
+    DeprecationWarning,
+    stacklevel=2,
 )
diff --git a/docs/apache-airflow-providers-core-sql/commits.rst b/docs/apache-airflow-providers-common-sql/commits.rst
similarity index 84%
rename from docs/apache-airflow-providers-core-sql/commits.rst
rename to docs/apache-airflow-providers-common-sql/commits.rst
index 8292a11950..b25291f136 100644
--- a/docs/apache-airflow-providers-core-sql/commits.rst
+++ b/docs/apache-airflow-providers-common-sql/commits.rst
@@ -15,11 +15,11 @@
     specific language governing permissions and limitations
     under the License.
 
-Package apache-airflow-providers-core-sql
-------------------------------------------
+Package apache-airflow-providers-common-sql
+-------------------------------------------
 
-`Core SQL Provider <https://en.wikipedia.org/wiki/SQL>`__
+`Common SQL Provider <https://en.wikipedia.org/wiki/SQL>`__
 
 
-This is detailed commit list of changes for versions provider package: ``core.sql``.
+This is detailed commit list of changes for versions provider package: ``common.sql``.
 For high-level changelog, see :doc:`package information including changelog <index>`.
diff --git a/docs/apache-airflow-providers-core-sql/connections.rst b/docs/apache-airflow-providers-common-sql/connections.rst
similarity index 100%
rename from docs/apache-airflow-providers-core-sql/connections.rst
rename to docs/apache-airflow-providers-common-sql/connections.rst
diff --git a/docs/apache-airflow-providers-core-sql/index.rst b/docs/apache-airflow-providers-common-sql/index.rst
similarity index 74%
rename from docs/apache-airflow-providers-core-sql/index.rst
rename to docs/apache-airflow-providers-common-sql/index.rst
index abc6186b68..c1f8ddbc9f 100644
--- a/docs/apache-airflow-providers-core-sql/index.rst
+++ b/docs/apache-airflow-providers-common-sql/index.rst
@@ -15,8 +15,8 @@
     specific language governing permissions and limitations
     under the License.
 
-``apache-airflow-providers-core-sql``
-============================================
+``apache-airflow-providers-common-sql``
+=======================================
 
 Content
 -------
@@ -32,14 +32,20 @@ Content
     :maxdepth: 1
     :caption: References
 
-    Python API <_api/airflow/providers/core/sql/index>
+    Python API <_api/airflow/providers/common/sql/index>
+
+.. toctree::
+    :hidden:
+    :caption: System tests
+
+    System Tests <_api/tests/system/providers/common/sql/index>
 
 .. toctree::
     :maxdepth: 1
     :caption: Resources
 
-    Example DAGs <https://github.com/apache/airflow/tree/main/airflow/providers/core/sql/example_dags>
-    PyPI Repository <https://pypi.org/project/apache-airflow-providers-core-sql/>
+    Example DAGs <https://github.com/apache/airflow/tree/providers-common-sql/1.0.0/tests/system/providers/common/sql>
+    PyPI Repository <https://pypi.org/project/apache-airflow-providers-common-sql/>
     Installing from sources <installing-providers-from-sources>
 
 .. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME!
@@ -50,5 +56,5 @@ Content
     Detailed list of commits <commits>
 
 
-Package apache-airflow-providers-core-sql
-------------------------------------------------------
+Package apache-airflow-providers-common-sql
+-------------------------------------------
diff --git a/docs/apache-airflow-providers-core-sql/installing-providers-from-sources.rst b/docs/apache-airflow-providers-common-sql/installing-providers-from-sources.rst
similarity index 100%
rename from docs/apache-airflow-providers-core-sql/installing-providers-from-sources.rst
rename to docs/apache-airflow-providers-common-sql/installing-providers-from-sources.rst
diff --git a/docs/apache-airflow-providers-core-sql/operators.rst b/docs/apache-airflow-providers-common-sql/operators.rst
similarity index 90%
rename from docs/apache-airflow-providers-core-sql/operators.rst
rename to docs/apache-airflow-providers-common-sql/operators.rst
index c2b04767d4..039e10b966 100644
--- a/docs/apache-airflow-providers-core-sql/operators.rst
+++ b/docs/apache-airflow-providers-common-sql/operators.rst
@@ -16,7 +16,7 @@
     under the License.
 
 SQL Operators
-===================
+=============
 
 These operators perform various queries against a SQL database, including
 column- and table-level data quality checks.
@@ -26,7 +26,7 @@ column- and table-level data quality checks.
 Check SQL Table Columns
 ~~~~~~~~~~~~~~~~~~~~~~~
 
-Use the :class:`~airflow.providers.core.sql.operators.sql.SQLColumnCheckOperator` to run data quality
+Use the :class:`~airflow.providers.common.sql.operators.sql.SQLColumnCheckOperator` to run data quality
 checks against columns of a given table. As well as a connection ID and table, a column_mapping
 describing the relationship between columns and tests to run must be supplied. An example column
 mapping is a set of three nested dictionaries and looks like:
@@ -71,7 +71,7 @@ be out of bounds but still considered successful.
 
 The below example demonstrates how to instantiate the SQLColumnCheckOperator task.
 
-.. exampleinclude:: /../../airflow/providers/core/sql/example_dags/example_sql_column_table_check.py
+.. exampleinclude:: /../../tests/system/providers/common/sql/example_sql_column_table_check.py
     :language: python
     :dedent: 4
     :start-after: [START howto_operator_sql_column_check]
@@ -82,7 +82,7 @@ The below example demonstrates how to instantiate the SQLColumnCheckOperator tas
 Check SQL Table Values
 ~~~~~~~~~~~~~~~~~~~~~~~
 
-Use the :class:`~airflow.providers.core.sql.operators.sql.SQLTableCheckOperator` to run data quality
+Use the :class:`~airflow.providers.common.sql.operators.sql.SQLTableCheckOperator` to run data quality
 checks against a given table. As well as a connection ID and table, a checks dictionary
 describing the relationship between the table and tests to run must be supplied. An example
 checks argument is a set of two nested dictionaries and looks like:
@@ -105,7 +105,7 @@ airflow.operators.sql.parse_boolean).
 
 The below example demonstrates how to instantiate the SQLTableCheckOperator task.
 
-.. exampleinclude:: /../../airflow/providers/core/sql/example_dags/example_sql_column_table_check.py
+.. exampleinclude:: /../../tests/system/providers/common/sql/example_sql_column_table_check.py
     :language: python
     :dedent: 4
     :start-after: [START howto_operator_sql_table_check]
diff --git a/docs/apache-airflow-providers-odbc/connections/odbc.rst b/docs/apache-airflow-providers-odbc/connections/odbc.rst
index 5d715596aa..176977d8fe 100644
--- a/docs/apache-airflow-providers-odbc/connections/odbc.rst
+++ b/docs/apache-airflow-providers-odbc/connections/odbc.rst
@@ -65,7 +65,7 @@ Extra (optional)
         * key-value pairs under ``connect_kwargs`` will be passed onto ``pyodbc.connect`` as kwargs
     - ``sqlalchemy_scheme``
         * This is only used when ``get_uri`` is invoked in
-          :py:meth:`~airflow.hooks.dbapi.DbApiHook.get_sqlalchemy_engine`.  By default, the hook uses
+          :py:meth:`~airflow.providers.common.sql.hooks.sql.DbApiHook.get_sqlalchemy_engine`.  By default, the hook uses
           scheme ``mssql+pyodbc``.  You may pass a string value here to override.
 
     .. note::
diff --git a/docs/apache-airflow/extra-packages-ref.rst b/docs/apache-airflow/extra-packages-ref.rst
index 0e7094ce17..a79b5625e7 100644
--- a/docs/apache-airflow/extra-packages-ref.rst
+++ b/docs/apache-airflow/extra-packages-ref.rst
@@ -270,7 +270,7 @@ These are extras that provide support for integration with external systems via
 +---------------------+-----------------------------------------------------+--------------------------------------+--------------+
 | extra               | install command                                     | enables                              | Preinstalled |
 +=====================+=====================================================+======================================+==============+
-| core.sql            | ``pip install 'apache-airflow[core.sql]'``          | Core SQL Operators                   |              |
+| common.sql          | ``pip install 'apache-airflow[common.sql]'``        | Core SQL Operators                   |      *       |
 +---------------------+-----------------------------------------------------+--------------------------------------+--------------+
 | ftp                 | ``pip install 'apache-airflow[ftp]'``               | FTP hooks and operators              |      *       |
 +---------------------+-----------------------------------------------------+--------------------------------------+--------------+
diff --git a/generated/provider_dependencies.json b/generated/provider_dependencies.json
index e14ee8af71..70ad0658fb 100644
--- a/generated/provider_dependencies.json
+++ b/generated/provider_dependencies.json
@@ -17,6 +17,7 @@
   },
   "amazon": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "boto3>=1.15.0",
       "jsonpath_ng>=1.5.3",
@@ -31,6 +32,7 @@
     "cross-providers-deps": [
       "apache.hive",
       "cncf.kubernetes",
+      "common.sql",
       "exasol",
       "ftp",
       "google",
@@ -58,19 +60,24 @@
   },
   "apache.drill": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "sqlalchemy-drill>=1.1.0",
       "sqlparse>=0.4.1"
     ],
-    "cross-providers-deps": []
+    "cross-providers-deps": [
+      "common.sql"
+    ]
   },
   "apache.druid": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "pydruid>=0.4.1"
     ],
     "cross-providers-deps": [
-      "apache.hive"
+      "apache.hive",
+      "common.sql"
     ]
   },
   "apache.hdfs": {
@@ -83,6 +90,7 @@
   },
   "apache.hive": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "hmsclient>=0.1.0",
       "pandas>=0.17.1",
@@ -92,6 +100,7 @@
     ],
     "cross-providers-deps": [
       "amazon",
+      "common.sql",
       "microsoft.mssql",
       "mysql",
       "presto",
@@ -123,10 +132,13 @@
   },
   "apache.pinot": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "pinotdb>0.1.2"
     ],
-    "cross-providers-deps": []
+    "cross-providers-deps": [
+      "common.sql"
+    ]
   },
   "apache.spark": {
     "deps": [
@@ -178,20 +190,21 @@
     ],
     "cross-providers-deps": []
   },
-  "core.sql": {
-    "deps": [
-      "apache-airflow>=2.2.0"
-    ],
+  "common.sql": {
+    "deps": [],
     "cross-providers-deps": []
   },
   "databricks": {
     "deps": [
       "aiohttp>=3.6.3, <4",
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "databricks-sql-connector>=2.0.0, <3.0.0",
       "requests>=2.27,<3"
     ],
-    "cross-providers-deps": []
+    "cross-providers-deps": [
+      "common.sql"
+    ]
   },
   "datadog": {
     "deps": [
@@ -236,20 +249,26 @@
   },
   "elasticsearch": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "elasticsearch-dbapi",
       "elasticsearch-dsl>=5.0.0",
       "elasticsearch>7"
     ],
-    "cross-providers-deps": []
+    "cross-providers-deps": [
+      "common.sql"
+    ]
   },
   "exasol": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "pandas>=0.17.1",
       "pyexasol>=0.5.1"
     ],
-    "cross-providers-deps": []
+    "cross-providers-deps": [
+      "common.sql"
+    ]
   },
   "facebook": {
     "deps": [
@@ -272,6 +291,7 @@
   "google": {
     "deps": [
       "PyOpenSSL",
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "google-ads>=15.1.1",
       "google-api-core>=2.7.0,<3.0.0",
@@ -322,6 +342,7 @@
       "apache.beam",
       "apache.cassandra",
       "cncf.kubernetes",
+      "common.sql",
       "facebook",
       "microsoft.azure",
       "microsoft.mssql",
@@ -373,10 +394,13 @@
   },
   "jdbc": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "jaydebeapi>=1.1.1"
     ],
-    "cross-providers-deps": []
+    "cross-providers-deps": [
+      "common.sql"
+    ]
   },
   "jenkins": {
     "deps": [
@@ -418,10 +442,13 @@
   },
   "microsoft.mssql": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "pymssql>=2.1.5; platform_machine != \"aarch64\""
     ],
-    "cross-providers-deps": []
+    "cross-providers-deps": [
+      "common.sql"
+    ]
   },
   "microsoft.psrp": {
     "deps": [
@@ -446,12 +473,14 @@
   },
   "mysql": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "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",
+      "common.sql",
       "presto",
       "trino",
       "vertica"
@@ -466,10 +495,13 @@
   },
   "odbc": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "pyodbc"
     ],
-    "cross-providers-deps": []
+    "cross-providers-deps": [
+      "common.sql"
+    ]
   },
   "openfaas": {
     "deps": [
@@ -486,10 +518,13 @@
   },
   "oracle": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "oracledb>=1.0.0"
     ],
-    "cross-providers-deps": []
+    "cross-providers-deps": [
+      "common.sql"
+    ]
   },
   "pagerduty": {
     "deps": [
@@ -515,30 +550,37 @@
   },
   "postgres": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "psycopg2-binary>=2.7.4"
     ],
     "cross-providers-deps": [
-      "amazon"
+      "amazon",
+      "common.sql"
     ]
   },
   "presto": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "pandas>=0.17.1",
       "presto-python-client>=0.8.2"
     ],
     "cross-providers-deps": [
+      "common.sql",
       "google",
       "slack"
     ]
   },
   "qubole": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "qds-sdk>=1.10.4"
     ],
-    "cross-providers-deps": []
+    "cross-providers-deps": [
+      "common.sql"
+    ]
   },
   "redis": {
     "deps": [
@@ -594,27 +636,35 @@
   },
   "slack": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow-providers-http",
       "apache-airflow>=2.2.0",
       "slack_sdk>=3.0.0"
     ],
     "cross-providers-deps": [
+      "common.sql",
       "http"
     ]
   },
   "snowflake": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "snowflake-connector-python>=2.4.1",
       "snowflake-sqlalchemy>=1.1.0"
     ],
     "cross-providers-deps": [
+      "common.sql",
       "slack"
     ]
   },
   "sqlite": {
-    "deps": [],
-    "cross-providers-deps": []
+    "deps": [
+      "apache-airflow-providers-common-sql"
+    ],
+    "cross-providers-deps": [
+      "common.sql"
+    ]
   },
   "ssh": {
     "deps": [
@@ -646,20 +696,25 @@
   },
   "trino": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "pandas>=0.17.1",
       "trino>=0.301.0"
     ],
     "cross-providers-deps": [
+      "common.sql",
       "google"
     ]
   },
   "vertica": {
     "deps": [
+      "apache-airflow-providers-common-sql",
       "apache-airflow>=2.2.0",
       "vertica-python>=0.5.1"
     ],
-    "cross-providers-deps": []
+    "cross-providers-deps": [
+      "common.sql"
+    ]
   },
   "yandex": {
     "deps": [
diff --git a/images/breeze/output-build-docs.svg b/images/breeze/output-build-docs.svg
index 50c566d055..5a05287b24 100644
--- a/images/breeze/output-build-docs.svg
+++ b/images/breeze/output-build-docs.svg
@@ -19,265 +19,265 @@
         font-weight: 700;
     }
 
-    .terminal-1768597619-matrix {
+    .terminal-3772557587-matrix {
         font-family: Fira Code, monospace;
         font-size: 20px;
         line-height: 24.4px;
         font-variant-east-asian: full-width;
     }
 
-    .terminal-1768597619-title {
+    .terminal-3772557587-title {
         font-size: 18px;
         font-weight: bold;
         font-family: arial;
     }
 
-    .terminal-1768597619-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-1768597619-r2 { fill: #c5c8c6 }
-.terminal-1768597619-r3 { fill: #d0b344;font-weight: bold }
-.terminal-1768597619-r4 { fill: #868887 }
-.terminal-1768597619-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-1768597619-r6 { fill: #98a84b;font-weight: bold }
-.terminal-1768597619-r7 { fill: #8d7b39 }
+    .terminal-3772557587-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-3772557587-r2 { fill: #c5c8c6 }
+.terminal-3772557587-r3 { fill: #d0b344;font-weight: bold }
+.terminal-3772557587-r4 { fill: #868887 }
+.terminal-3772557587-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-3772557587-r6 { fill: #98a84b;font-weight: bold }
+.terminal-3772557587-r7 { fill: #8d7b39 }
     </style>
 
     <defs>
-    <clipPath id="terminal-1768597619-clip-terminal">
+    <clipPath id="terminal-3772557587-clip-terminal">
       <rect x="0" y="0" width="1463.0" height="1365.3999999999999" />
     </clipPath>
-    <clipPath id="terminal-1768597619-line-0">
+    <clipPath id="terminal-3772557587-line-0">
     <rect x="0" y="1.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-1">
+<clipPath id="terminal-3772557587-line-1">
     <rect x="0" y="25.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-2">
+<clipPath id="terminal-3772557587-line-2">
     <rect x="0" y="50.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-3">
+<clipPath id="terminal-3772557587-line-3">
     <rect x="0" y="74.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-4">
+<clipPath id="terminal-3772557587-line-4">
     <rect x="0" y="99.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-5">
+<clipPath id="terminal-3772557587-line-5">
     <rect x="0" y="123.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-6">
+<clipPath id="terminal-3772557587-line-6">
     <rect x="0" y="147.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-7">
+<clipPath id="terminal-3772557587-line-7">
     <rect x="0" y="172.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-8">
+<clipPath id="terminal-3772557587-line-8">
     <rect x="0" y="196.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-9">
+<clipPath id="terminal-3772557587-line-9">
     <rect x="0" y="221.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-10">
+<clipPath id="terminal-3772557587-line-10">
     <rect x="0" y="245.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-11">
+<clipPath id="terminal-3772557587-line-11">
     <rect x="0" y="269.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-12">
+<clipPath id="terminal-3772557587-line-12">
     <rect x="0" y="294.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-13">
+<clipPath id="terminal-3772557587-line-13">
     <rect x="0" y="318.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-14">
+<clipPath id="terminal-3772557587-line-14">
     <rect x="0" y="343.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-15">
+<clipPath id="terminal-3772557587-line-15">
     <rect x="0" y="367.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-16">
+<clipPath id="terminal-3772557587-line-16">
     <rect x="0" y="391.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-17">
+<clipPath id="terminal-3772557587-line-17">
     <rect x="0" y="416.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-18">
+<clipPath id="terminal-3772557587-line-18">
     <rect x="0" y="440.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-19">
+<clipPath id="terminal-3772557587-line-19">
     <rect x="0" y="465.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-20">
+<clipPath id="terminal-3772557587-line-20">
     <rect x="0" y="489.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-21">
+<clipPath id="terminal-3772557587-line-21">
     <rect x="0" y="513.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-22">
+<clipPath id="terminal-3772557587-line-22">
     <rect x="0" y="538.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-23">
+<clipPath id="terminal-3772557587-line-23">
     <rect x="0" y="562.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-24">
+<clipPath id="terminal-3772557587-line-24">
     <rect x="0" y="587.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-25">
+<clipPath id="terminal-3772557587-line-25">
     <rect x="0" y="611.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-26">
+<clipPath id="terminal-3772557587-line-26">
     <rect x="0" y="635.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-27">
+<clipPath id="terminal-3772557587-line-27">
     <rect x="0" y="660.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-28">
+<clipPath id="terminal-3772557587-line-28">
     <rect x="0" y="684.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-29">
+<clipPath id="terminal-3772557587-line-29">
     <rect x="0" y="709.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-30">
+<clipPath id="terminal-3772557587-line-30">
     <rect x="0" y="733.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-31">
+<clipPath id="terminal-3772557587-line-31">
     <rect x="0" y="757.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-32">
+<clipPath id="terminal-3772557587-line-32">
     <rect x="0" y="782.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-33">
+<clipPath id="terminal-3772557587-line-33">
     <rect x="0" y="806.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-34">
+<clipPath id="terminal-3772557587-line-34">
     <rect x="0" y="831.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-35">
+<clipPath id="terminal-3772557587-line-35">
     <rect x="0" y="855.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-36">
+<clipPath id="terminal-3772557587-line-36">
     <rect x="0" y="879.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-37">
+<clipPath id="terminal-3772557587-line-37">
     <rect x="0" y="904.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-38">
+<clipPath id="terminal-3772557587-line-38">
     <rect x="0" y="928.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-39">
+<clipPath id="terminal-3772557587-line-39">
     <rect x="0" y="953.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-40">
+<clipPath id="terminal-3772557587-line-40">
     <rect x="0" y="977.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-41">
+<clipPath id="terminal-3772557587-line-41">
     <rect x="0" y="1001.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-42">
+<clipPath id="terminal-3772557587-line-42">
     <rect x="0" y="1026.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-43">
+<clipPath id="terminal-3772557587-line-43">
     <rect x="0" y="1050.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-44">
+<clipPath id="terminal-3772557587-line-44">
     <rect x="0" y="1075.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-45">
+<clipPath id="terminal-3772557587-line-45">
     <rect x="0" y="1099.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-46">
+<clipPath id="terminal-3772557587-line-46">
     <rect x="0" y="1123.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-47">
+<clipPath id="terminal-3772557587-line-47">
     <rect x="0" y="1148.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-48">
+<clipPath id="terminal-3772557587-line-48">
     <rect x="0" y="1172.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-49">
+<clipPath id="terminal-3772557587-line-49">
     <rect x="0" y="1197.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-50">
+<clipPath id="terminal-3772557587-line-50">
     <rect x="0" y="1221.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-51">
+<clipPath id="terminal-3772557587-line-51">
     <rect x="0" y="1245.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-52">
+<clipPath id="terminal-3772557587-line-52">
     <rect x="0" y="1270.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-53">
+<clipPath id="terminal-3772557587-line-53">
     <rect x="0" y="1294.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1768597619-line-54">
+<clipPath id="terminal-3772557587-line-54">
     <rect x="0" y="1319.1" 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="1414.4" rx="8"/><text class="terminal-1768597619-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;build-docs</text>
+    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="1414.4" rx="8"/><text class="terminal-3772557587-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;build-docs</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-1768597619-clip-terminal)">
+    <g transform="translate(9, 41)" clip-path="url(#terminal-3772557587-clip-terminal)">
     
-    <g class="terminal-1768597619-matrix">
-    <text class="terminal-1768597619-r2" x="1464" y="20" textLength="12.2" clip-path="url(#terminal-1768597619-line-0)">
-</text><text class="terminal-1768597619-r3" x="12.2" y="44.4" textLength="85.4" clip-path="url(#terminal-1768597619-line-1)">Usage:&#160;</text><text class="terminal-1768597619-r1" x="97.6" y="44.4" textLength="329.4" clip-path="url(#terminal-1768597619-line-1)">breeze&#160;build-docs&#160;[OPTIONS]</text><text class="terminal-1768597619-r2" x="1464" y="44.4" textLength="12.2" clip-path="url(#terminal-1768597619-line-1)">
-</text><text class="terminal-1768597619-r2" x="1464" y="68.8" textLength="12.2" clip-path="url(#terminal-1768597619-line-2)">
-</text><text class="terminal-1768597619-r2" x="12.2" y="93.2" textLength="451.4" clip-path="url(#terminal-1768597619-line-3)">Build&#160;documentation&#160;in&#160;the&#160;container.</text><text class="terminal-1768597619-r2" x="1464" y="93.2" textLength="12.2" clip-path="url(#terminal-1768597619-line-3)">
-</text><text class="terminal-1768597619-r2" x="1464" y="117.6" textLength="12.2" clip-path="url(#terminal-1768597619-line-4)">
-</text><text class="terminal-1768597619-r4" x="0" y="142" textLength="24.4" clip-path="url(#terminal-1768597619-line-5)">╭─</text><text class="terminal-1768597619-r4" x="24.4" y="142" textLength="1415.2" clip-path="url(#terminal-1768597619-line-5)">&#160;Doc&#160;flags&#160;─────────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-1768597619-r4" x="1439.6" y="142" textLength="24.4" clip-path="url(#terminal-176859 [...]
-</text><text class="terminal-1768597619-r4" x="0" y="166.4" textLength="12.2" clip-path="url(#terminal-1768597619-line-6)">│</text><text class="terminal-1768597619-r5" x="24.4" y="166.4" textLength="12.2" clip-path="url(#terminal-1768597619-line-6)">-</text><text class="terminal-1768597619-r5" x="36.6" y="166.4" textLength="61" clip-path="url(#terminal-1768597619-line-6)">-docs</text><text class="terminal-1768597619-r5" x="97.6" y="166.4" textLength="61" clip-path="url(#terminal-17685976 [...]
-</text><text class="terminal-1768597619-r4" x="0" y="190.8" textLength="12.2" clip-path="url(#terminal-1768597619-line-7)">│</text><text class="terminal-1768597619-r5" x="24.4" y="190.8" textLength="12.2" clip-path="url(#terminal-1768597619-line-7)">-</text><text class="terminal-1768597619-r5" x="36.6" y="190.8" textLength="134.2" clip-path="url(#terminal-1768597619-line-7)">-spellcheck</text><text class="terminal-1768597619-r5" x="170.8" y="190.8" textLength="61" clip-path="url(#termina [...]
-</text><text class="terminal-1768597619-r4" x="0" y="215.2" textLength="12.2" clip-path="url(#terminal-1768597619-line-8)">│</text><text class="terminal-1768597619-r5" x="24.4" y="215.2" textLength="12.2" clip-path="url(#terminal-1768597619-line-8)">-</text><text class="terminal-1768597619-r5" x="36.6" y="215.2" textLength="48.8" clip-path="url(#terminal-1768597619-line-8)">-for</text><text class="terminal-1768597619-r5" x="85.4" y="215.2" textLength="134.2" clip-path="url(#terminal-1768 [...]
-</text><text class="terminal-1768597619-r4" x="0" y="239.6" textLength="12.2" clip-path="url(#terminal-1768597619-line-9)">│</text><text class="terminal-1768597619-r5" x="24.4" y="239.6" textLength="12.2" clip-path="url(#terminal-1768597619-line-9)">-</text><text class="terminal-1768597619-r5" x="36.6" y="239.6" textLength="97.6" clip-path="url(#terminal-1768597619-line-9)">-package</text><text class="terminal-1768597619-r5" x="134.2" y="239.6" textLength="85.4" clip-path="url(#terminal- [...]
-</text><text class="terminal-1768597619-r4" x="0" y="264" textLength="12.2" clip-path="url(#terminal-1768597619-line-10)">│</text><text class="terminal-1768597619-r7" x="305" y="264" textLength="1134.6" clip-path="url(#terminal-1768597619-line-10)">(apache-airflow&#160;|&#160;apache-airflow-providers&#160;|&#160;apache-airflow-providers-airbyte&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1768597619-r4" x="1451.8"  [...]
-</text><text class="terminal-1768597619-r4" x="0" y="288.4" textLength="12.2" clip-path="url(#terminal-1768597619-line-11)">│</text><text class="terminal-1768597619-r7" x="305" y="288.4" textLength="1134.6" clip-path="url(#terminal-1768597619-line-11)">apache-airflow-providers-alibaba&#160;|&#160;apache-airflow-providers-amazon&#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><text class="terminal-1768597619-r4" x="0" y="312.8" textLength="12.2" clip-path="url(#terminal-1768597619-line-12)">│</text><text class="terminal-1768597619-r7" x="305" y="312.8" textLength="1134.6" clip-path="url(#terminal-1768597619-line-12)">apache-airflow-providers-apache-beam&#160;|&#160;apache-airflow-providers-apache-cassandra&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1768597619-r4" x="1451.8" y="312.8" textLength= [...]
-</text><text class="terminal-1768597619-r4" x="0" y="337.2" textLength="12.2" clip-path="url(#terminal-1768597619-line-13)">│</text><text class="terminal-1768597619-r7" x="305" y="337.2" textLength="1134.6" clip-path="url(#terminal-1768597619-line-13)">apache-airflow-providers-apache-drill&#160;|&#160;apache-airflow-providers-apache-druid&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1768597619-r4" x="1451.8" y="337 [...]
-</text><text class="terminal-1768597619-r4" x="0" y="361.6" textLength="12.2" clip-path="url(#terminal-1768597619-line-14)">│</text><text class="terminal-1768597619-r7" x="305" y="361.6" textLength="1134.6" clip-path="url(#terminal-1768597619-line-14)">apache-airflow-providers-apache-hdfs&#160;|&#160;apache-airflow-providers-apache-hive&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1768597619-r4" x="1451 [...]
-</text><text class="terminal-1768597619-r4" x="0" y="386" textLength="12.2" clip-path="url(#terminal-1768597619-line-15)">│</text><text class="terminal-1768597619-r7" x="305" y="386" textLength="1134.6" clip-path="url(#terminal-1768597619-line-15)">apache-airflow-providers-apache-kylin&#160;|&#160;apache-airflow-providers-apache-livy&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1768597619-r4" x="1451.8" y="38 [...]
-</text><text class="terminal-1768597619-r4" x="0" y="410.4" textLength="12.2" clip-path="url(#terminal-1768597619-line-16)">│</text><text class="terminal-1768597619-r7" x="305" y="410.4" textLength="1134.6" clip-path="url(#terminal-1768597619-line-16)">apache-airflow-providers-apache-pig&#160;|&#160;apache-airflow-providers-apache-pinot&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1768597619-r4" x="1451 [...]
-</text><text class="terminal-1768597619-r4" x="0" y="434.8" textLength="12.2" clip-path="url(#terminal-1768597619-line-17)">│</text><text class="terminal-1768597619-r7" x="305" y="434.8" textLength="1134.6" clip-path="url(#terminal-1768597619-line-17)">apache-airflow-providers-apache-spark&#160;|&#160;apache-airflow-providers-apache-sqoop&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1768597619-r4" x="1451.8" y="434 [...]
-</text><text class="terminal-1768597619-r4" x="0" y="459.2" textLength="12.2" clip-path="url(#terminal-1768597619-line-18)">│</text><text class="terminal-1768597619-r7" x="305" y="459.2" textLength="1134.6" clip-path="url(#terminal-1768597619-line-18)">apache-airflow-providers-arangodb&#160;|&#160;apache-airflow-providers-asana&#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><text class="terminal-1768597619-r4" x="0" y="483.6" textLength="12.2" clip-path="url(#terminal-1768597619-line-19)">│</text><text class="terminal-1768597619-r7" x="305" y="483.6" textLength="1134.6" clip-path="url(#terminal-1768597619-line-19)">apache-airflow-providers-celery&#160;|&#160;apache-airflow-providers-cloudant&#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><tex [...]
-</text><text class="terminal-1768597619-r4" x="0" y="508" textLength="12.2" clip-path="url(#terminal-1768597619-line-20)">│</text><text class="terminal-1768597619-r7" x="305" y="508" textLength="1134.6" clip-path="url(#terminal-1768597619-line-20)">apache-airflow-providers-cncf-kubernetes&#160;|&#160;apache-airflow-providers-core-sql&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1768597619-r4" x="1451.8" y="50 [...]
-</text><text class="terminal-1768597619-r4" x="0" y="532.4" textLength="12.2" clip-path="url(#terminal-1768597619-line-21)">│</text><text class="terminal-1768597619-r7" x="305" y="532.4" textLength="1134.6" clip-path="url(#terminal-1768597619-line-21)">apache-airflow-providers-databricks&#160;|&#160;apache-airflow-providers-datadog&#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-1768597619-r4" x="0" y="556.8" textLength="12.2" clip-path="url(#terminal-1768597619-line-22)">│</text><text class="terminal-1768597619-r7" x="305" y="556.8" textLength="1134.6" clip-path="url(#terminal-1768597619-line-22)">apache-airflow-providers-dbt-cloud&#160;|&#160;apache-airflow-providers-dingding&#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-1768597619-r4" x="0" y="581.2" textLength="12.2" clip-path="url(#terminal-1768597619-line-23)">│</text><text class="terminal-1768597619-r7" x="305" y="581.2" textLength="1134.6" clip-path="url(#terminal-1768597619-line-23)">apache-airflow-providers-discord&#160;|&#160;apache-airflow-providers-docker&#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><text class="terminal-1768597619-r4" x="0" y="605.6" textLength="12.2" clip-path="url(#terminal-1768597619-line-24)">│</text><text class="terminal-1768597619-r7" x="305" y="605.6" textLength="1134.6" clip-path="url(#terminal-1768597619-line-24)">apache-airflow-providers-elasticsearch&#160;|&#160;apache-airflow-providers-exasol&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1768597 [...]
-</text><text class="terminal-1768597619-r4" x="0" y="630" textLength="12.2" clip-path="url(#terminal-1768597619-line-25)">│</text><text class="terminal-1768597619-r7" x="305" y="630" textLength="1134.6" clip-path="url(#terminal-1768597619-line-25)">apache-airflow-providers-facebook&#160;|&#160;apache-airflow-providers-ftp&#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-1768597619-r4" x="0" y="654.4" textLength="12.2" clip-path="url(#terminal-1768597619-line-26)">│</text><text class="terminal-1768597619-r7" x="305" y="654.4" textLength="1134.6" clip-path="url(#terminal-1768597619-line-26)">apache-airflow-providers-github&#160;|&#160;apache-airflow-providers-google&#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-1768597619-r4" x="0" y="678.8" textLength="12.2" clip-path="url(#terminal-1768597619-line-27)">│</text><text class="terminal-1768597619-r7" x="305" y="678.8" textLength="1134.6" clip-path="url(#terminal-1768597619-line-27)">apache-airflow-providers-grpc&#160;|&#160;apache-airflow-providers-hashicorp&#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><text class="terminal-1768597619-r4" x="0" y="703.2" textLength="12.2" clip-path="url(#terminal-1768597619-line-28)">│</text><text class="terminal-1768597619-r7" x="305" y="703.2" textLength="1134.6" clip-path="url(#terminal-1768597619-line-28)">apache-airflow-providers-http&#160;|&#160;apache-airflow-providers-imap&#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-1768597619-r4" x="0" y="727.6" textLength="12.2" clip-path="url(#terminal-1768597619-line-29)">│</text><text class="terminal-1768597619-r7" x="305" y="727.6" textLength="1134.6" clip-path="url(#terminal-1768597619-line-29)">apache-airflow-providers-influxdb&#160;|&#160;apache-airflow-providers-jdbc&#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-1768597619-r4" x="0" y="752" textLength="12.2" clip-path="url(#terminal-1768597619-line-30)">│</text><text class="terminal-1768597619-r7" x="305" y="752" textLength="1134.6" clip-path="url(#terminal-1768597619-line-30)">apache-airflow-providers-jenkins&#160;|&#160;apache-airflow-providers-jira&#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-1768597619-r4" x="0" y="776.4" textLength="12.2" clip-path="url(#terminal-1768597619-line-31)">│</text><text class="terminal-1768597619-r7" x="305" y="776.4" textLength="1134.6" clip-path="url(#terminal-1768597619-line-31)">apache-airflow-providers-microsoft-azure&#160;|&#160;apache-airflow-providers-microsoft-mssql&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1768597619-r4" x="1451.8" y="776.4" textLength="12.2" clip-pat [...]
-</text><text class="terminal-1768597619-r4" x="0" y="800.8" textLength="12.2" clip-path="url(#terminal-1768597619-line-32)">│</text><text class="terminal-1768597619-r7" x="305" y="800.8" textLength="1134.6" clip-path="url(#terminal-1768597619-line-32)">apache-airflow-providers-microsoft-psrp&#160;|&#160;apache-airflow-providers-microsoft-winrm&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1768597619-r4" x="1451.8" y="800.8" textLength="12.2" cli [...]
-</text><text class="terminal-1768597619-r4" x="0" y="825.2" textLength="12.2" clip-path="url(#terminal-1768597619-line-33)">│</text><text class="terminal-1768597619-r7" x="305" y="825.2" textLength="1134.6" clip-path="url(#terminal-1768597619-line-33)">apache-airflow-providers-mongo&#160;|&#160;apache-airflow-providers-mysql&#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-1768597619-r4" x="0" y="849.6" textLength="12.2" clip-path="url(#terminal-1768597619-line-34)">│</text><text class="terminal-1768597619-r7" x="305" y="849.6" textLength="1134.6" clip-path="url(#terminal-1768597619-line-34)">apache-airflow-providers-neo4j&#160;|&#160;apache-airflow-providers-odbc&#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-1768597619-r4" x="0" y="874" textLength="12.2" clip-path="url(#terminal-1768597619-line-35)">│</text><text class="terminal-1768597619-r7" x="305" y="874" textLength="1134.6" clip-path="url(#terminal-1768597619-line-35)">apache-airflow-providers-openfaas&#160;|&#160;apache-airflow-providers-opsgenie&#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="termi [...]
-</text><text class="terminal-1768597619-r4" x="0" y="898.4" textLength="12.2" clip-path="url(#terminal-1768597619-line-36)">│</text><text class="terminal-1768597619-r7" x="305" y="898.4" textLength="1134.6" clip-path="url(#terminal-1768597619-line-36)">apache-airflow-providers-oracle&#160;|&#160;apache-airflow-providers-pagerduty&#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-1768597619-r4" x="0" y="922.8" textLength="12.2" clip-path="url(#terminal-1768597619-line-37)">│</text><text class="terminal-1768597619-r7" x="305" y="922.8" textLength="1134.6" clip-path="url(#terminal-1768597619-line-37)">apache-airflow-providers-papermill&#160;|&#160;apache-airflow-providers-plexus&#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-1768597619-r4" x="0" y="947.2" textLength="12.2" clip-path="url(#terminal-1768597619-line-38)">│</text><text class="terminal-1768597619-r7" x="305" y="947.2" textLength="1134.6" clip-path="url(#terminal-1768597619-line-38)">apache-airflow-providers-postgres&#160;|&#160;apache-airflow-providers-presto&#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><tex [...]
-</text><text class="terminal-1768597619-r4" x="0" y="971.6" textLength="12.2" clip-path="url(#terminal-1768597619-line-39)">│</text><text class="terminal-1768597619-r7" x="305" y="971.6" textLength="1134.6" clip-path="url(#terminal-1768597619-line-39)">apache-airflow-providers-qubole&#160;|&#160;apache-airflow-providers-redis&#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-1768597619-r4" x="0" y="996" textLength="12.2" clip-path="url(#terminal-1768597619-line-40)">│</text><text class="terminal-1768597619-r7" x="305" y="996" textLength="1134.6" clip-path="url(#terminal-1768597619-line-40)">apache-airflow-providers-salesforce&#160;|&#160;apache-airflow-providers-samba&#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=" [...]
-</text><text class="terminal-1768597619-r4" x="0" y="1020.4" textLength="12.2" clip-path="url(#terminal-1768597619-line-41)">│</text><text class="terminal-1768597619-r7" x="305" y="1020.4" textLength="1134.6" clip-path="url(#terminal-1768597619-line-41)">apache-airflow-providers-segment&#160;|&#160;apache-airflow-providers-sendgrid&#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 c [...]
-</text><text class="terminal-1768597619-r4" x="0" y="1044.8" textLength="12.2" clip-path="url(#terminal-1768597619-line-42)">│</text><text class="terminal-1768597619-r7" x="305" y="1044.8" textLength="1134.6" clip-path="url(#terminal-1768597619-line-42)">apache-airflow-providers-sftp&#160;|&#160;apache-airflow-providers-singularity&#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 c [...]
-</text><text class="terminal-1768597619-r4" x="0" y="1069.2" textLength="12.2" clip-path="url(#terminal-1768597619-line-43)">│</text><text class="terminal-1768597619-r7" x="305" y="1069.2" textLength="1134.6" clip-path="url(#terminal-1768597619-line-43)">apache-airflow-providers-slack&#160;|&#160;apache-airflow-providers-snowflake&#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><t [...]
-</text><text class="terminal-1768597619-r4" x="0" y="1093.6" textLength="12.2" clip-path="url(#terminal-1768597619-line-44)">│</text><text class="terminal-1768597619-r7" x="305" y="1093.6" textLength="1134.6" clip-path="url(#terminal-1768597619-line-44)">apache-airflow-providers-sqlite&#160;|&#160;apache-airflow-providers-ssh&#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-1768597619-r4" x="0" y="1118" textLength="12.2" clip-path="url(#terminal-1768597619-line-45)">│</text><text class="terminal-1768597619-r7" x="305" y="1118" textLength="1134.6" clip-path="url(#terminal-1768597619-line-45)">apache-airflow-providers-tableau&#160;|&#160;apache-airflow-providers-tabular&#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  [...]
-</text><text class="terminal-1768597619-r4" x="0" y="1142.4" textLength="12.2" clip-path="url(#terminal-1768597619-line-46)">│</text><text class="terminal-1768597619-r7" x="305" y="1142.4" textLength="1134.6" clip-path="url(#terminal-1768597619-line-46)">apache-airflow-providers-telegram&#160;|&#160;apache-airflow-providers-trino&#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;</te [...]
-</text><text class="terminal-1768597619-r4" x="0" y="1166.8" textLength="12.2" clip-path="url(#terminal-1768597619-line-47)">│</text><text class="terminal-1768597619-r7" x="305" y="1166.8" textLength="1134.6" clip-path="url(#terminal-1768597619-line-47)">apache-airflow-providers-vertica&#160;|&#160;apache-airflow-providers-yandex&#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;</te [...]
-</text><text class="terminal-1768597619-r4" x="0" y="1191.2" textLength="12.2" clip-path="url(#terminal-1768597619-line-48)">│</text><text class="terminal-1768597619-r7" x="305" y="1191.2" textLength="1134.6" clip-path="url(#terminal-1768597619-line-48)">apache-airflow-providers-zendesk&#160;|&#160;docker-stack&#160;|&#160;helm-chart)&#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-1768597619-r4" x="0" y="1215.6" textLength="1464" clip-path="url(#terminal-1768597619-line-49)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1768597619-r2" x="1464" y="1215.6" textLength="12.2" clip-path="url(#terminal-1768597619-line-49)">
-</text><text class="terminal-1768597619-r4" x="0" y="1240" textLength="24.4" clip-path="url(#terminal-1768597619-line-50)">╭─</text><text class="terminal-1768597619-r4" x="24.4" y="1240" textLength="1415.2" clip-path="url(#terminal-1768597619-line-50)">&#160;Options&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-1768597619-r4" x="1439.6" y="1240" textLength="24.4" clip-path="url(#terminal-176859 [...]
-</text><text class="terminal-1768597619-r4" x="0" y="1264.4" textLength="12.2" clip-path="url(#terminal-1768597619-line-51)">│</text><text class="terminal-1768597619-r5" x="24.4" y="1264.4" textLength="12.2" clip-path="url(#terminal-1768597619-line-51)">-</text><text class="terminal-1768597619-r5" x="36.6" y="1264.4" textLength="97.6" clip-path="url(#terminal-1768597619-line-51)">-verbose</text><text class="terminal-1768597619-r6" x="280.6" y="1264.4" textLength="24.4" clip-path="url(#te [...]
-</text><text class="terminal-1768597619-r4" x="0" y="1288.8" textLength="12.2" clip-path="url(#terminal-1768597619-line-52)">│</text><text class="terminal-1768597619-r5" x="24.4" y="1288.8" textLength="12.2" clip-path="url(#terminal-1768597619-line-52)">-</text><text class="terminal-1768597619-r5" x="36.6" y="1288.8" textLength="48.8" clip-path="url(#terminal-1768597619-line-52)">-dry</text><text class="terminal-1768597619-r5" x="85.4" y="1288.8" textLength="48.8" clip-path="url(#termina [...]
-</text><text class="terminal-1768597619-r4" x="0" y="1313.2" textLength="12.2" clip-path="url(#terminal-1768597619-line-53)">│</text><text class="terminal-1768597619-r5" x="24.4" y="1313.2" textLength="12.2" clip-path="url(#terminal-1768597619-line-53)">-</text><text class="terminal-1768597619-r5" x="36.6" y="1313.2" textLength="85.4" clip-path="url(#terminal-1768597619-line-53)">-github</text><text class="terminal-1768597619-r5" x="122" y="1313.2" textLength="134.2" clip-path="url(#term [...]
-</text><text class="terminal-1768597619-r4" x="0" y="1337.6" textLength="12.2" clip-path="url(#terminal-1768597619-line-54)">│</text><text class="terminal-1768597619-r5" x="24.4" y="1337.6" textLength="12.2" clip-path="url(#terminal-1768597619-line-54)">-</text><text class="terminal-1768597619-r5" x="36.6" y="1337.6" textLength="61" clip-path="url(#terminal-1768597619-line-54)">-help</text><text class="terminal-1768597619-r6" x="280.6" y="1337.6" textLength="24.4" clip-path="url(#termina [...]
-</text><text class="terminal-1768597619-r4" x="0" y="1362" textLength="1464" clip-path="url(#terminal-1768597619-line-55)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1768597619-r2" x="1464" y="1362" textLength="12.2" clip-path="url(#terminal-1768597619-line-55)">
+    <g class="terminal-3772557587-matrix">
+    <text class="terminal-3772557587-r2" x="1464" y="20" textLength="12.2" clip-path="url(#terminal-3772557587-line-0)">
+</text><text class="terminal-3772557587-r3" x="12.2" y="44.4" textLength="85.4" clip-path="url(#terminal-3772557587-line-1)">Usage:&#160;</text><text class="terminal-3772557587-r1" x="97.6" y="44.4" textLength="329.4" clip-path="url(#terminal-3772557587-line-1)">breeze&#160;build-docs&#160;[OPTIONS]</text><text class="terminal-3772557587-r2" x="1464" y="44.4" textLength="12.2" clip-path="url(#terminal-3772557587-line-1)">
+</text><text class="terminal-3772557587-r2" x="1464" y="68.8" textLength="12.2" clip-path="url(#terminal-3772557587-line-2)">
+</text><text class="terminal-3772557587-r2" x="12.2" y="93.2" textLength="451.4" clip-path="url(#terminal-3772557587-line-3)">Build&#160;documentation&#160;in&#160;the&#160;container.</text><text class="terminal-3772557587-r2" x="1464" y="93.2" textLength="12.2" clip-path="url(#terminal-3772557587-line-3)">
+</text><text class="terminal-3772557587-r2" x="1464" y="117.6" textLength="12.2" clip-path="url(#terminal-3772557587-line-4)">
+</text><text class="terminal-3772557587-r4" x="0" y="142" textLength="24.4" clip-path="url(#terminal-3772557587-line-5)">╭─</text><text class="terminal-3772557587-r4" x="24.4" y="142" textLength="1415.2" clip-path="url(#terminal-3772557587-line-5)">&#160;Doc&#160;flags&#160;─────────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-3772557587-r4" x="1439.6" y="142" textLength="24.4" clip-path="url(#terminal-377255 [...]
+</text><text class="terminal-3772557587-r4" x="0" y="166.4" textLength="12.2" clip-path="url(#terminal-3772557587-line-6)">│</text><text class="terminal-3772557587-r5" x="24.4" y="166.4" textLength="12.2" clip-path="url(#terminal-3772557587-line-6)">-</text><text class="terminal-3772557587-r5" x="36.6" y="166.4" textLength="61" clip-path="url(#terminal-3772557587-line-6)">-docs</text><text class="terminal-3772557587-r5" x="97.6" y="166.4" textLength="61" clip-path="url(#terminal-37725575 [...]
+</text><text class="terminal-3772557587-r4" x="0" y="190.8" textLength="12.2" clip-path="url(#terminal-3772557587-line-7)">│</text><text class="terminal-3772557587-r5" x="24.4" y="190.8" textLength="12.2" clip-path="url(#terminal-3772557587-line-7)">-</text><text class="terminal-3772557587-r5" x="36.6" y="190.8" textLength="134.2" clip-path="url(#terminal-3772557587-line-7)">-spellcheck</text><text class="terminal-3772557587-r5" x="170.8" y="190.8" textLength="61" clip-path="url(#termina [...]
+</text><text class="terminal-3772557587-r4" x="0" y="215.2" textLength="12.2" clip-path="url(#terminal-3772557587-line-8)">│</text><text class="terminal-3772557587-r5" x="24.4" y="215.2" textLength="12.2" clip-path="url(#terminal-3772557587-line-8)">-</text><text class="terminal-3772557587-r5" x="36.6" y="215.2" textLength="48.8" clip-path="url(#terminal-3772557587-line-8)">-for</text><text class="terminal-3772557587-r5" x="85.4" y="215.2" textLength="134.2" clip-path="url(#terminal-3772 [...]
+</text><text class="terminal-3772557587-r4" x="0" y="239.6" textLength="12.2" clip-path="url(#terminal-3772557587-line-9)">│</text><text class="terminal-3772557587-r5" x="24.4" y="239.6" textLength="12.2" clip-path="url(#terminal-3772557587-line-9)">-</text><text class="terminal-3772557587-r5" x="36.6" y="239.6" textLength="97.6" clip-path="url(#terminal-3772557587-line-9)">-package</text><text class="terminal-3772557587-r5" x="134.2" y="239.6" textLength="85.4" clip-path="url(#terminal- [...]
+</text><text class="terminal-3772557587-r4" x="0" y="264" textLength="12.2" clip-path="url(#terminal-3772557587-line-10)">│</text><text class="terminal-3772557587-r7" x="305" y="264" textLength="1134.6" clip-path="url(#terminal-3772557587-line-10)">(apache-airflow&#160;|&#160;apache-airflow-providers&#160;|&#160;apache-airflow-providers-airbyte&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-3772557587-r4" x="1451.8"  [...]
+</text><text class="terminal-3772557587-r4" x="0" y="288.4" textLength="12.2" clip-path="url(#terminal-3772557587-line-11)">│</text><text class="terminal-3772557587-r7" x="305" y="288.4" textLength="1134.6" clip-path="url(#terminal-3772557587-line-11)">apache-airflow-providers-alibaba&#160;|&#160;apache-airflow-providers-amazon&#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><text class="terminal-3772557587-r4" x="0" y="312.8" textLength="12.2" clip-path="url(#terminal-3772557587-line-12)">│</text><text class="terminal-3772557587-r7" x="305" y="312.8" textLength="1134.6" clip-path="url(#terminal-3772557587-line-12)">apache-airflow-providers-apache-beam&#160;|&#160;apache-airflow-providers-apache-cassandra&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-3772557587-r4" x="1451.8" y="312.8" textLength= [...]
+</text><text class="terminal-3772557587-r4" x="0" y="337.2" textLength="12.2" clip-path="url(#terminal-3772557587-line-13)">│</text><text class="terminal-3772557587-r7" x="305" y="337.2" textLength="1134.6" clip-path="url(#terminal-3772557587-line-13)">apache-airflow-providers-apache-drill&#160;|&#160;apache-airflow-providers-apache-druid&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-3772557587-r4" x="1451.8" y="337 [...]
+</text><text class="terminal-3772557587-r4" x="0" y="361.6" textLength="12.2" clip-path="url(#terminal-3772557587-line-14)">│</text><text class="terminal-3772557587-r7" x="305" y="361.6" textLength="1134.6" clip-path="url(#terminal-3772557587-line-14)">apache-airflow-providers-apache-hdfs&#160;|&#160;apache-airflow-providers-apache-hive&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-3772557587-r4" x="1451 [...]
+</text><text class="terminal-3772557587-r4" x="0" y="386" textLength="12.2" clip-path="url(#terminal-3772557587-line-15)">│</text><text class="terminal-3772557587-r7" x="305" y="386" textLength="1134.6" clip-path="url(#terminal-3772557587-line-15)">apache-airflow-providers-apache-kylin&#160;|&#160;apache-airflow-providers-apache-livy&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-3772557587-r4" x="1451.8" y="38 [...]
+</text><text class="terminal-3772557587-r4" x="0" y="410.4" textLength="12.2" clip-path="url(#terminal-3772557587-line-16)">│</text><text class="terminal-3772557587-r7" x="305" y="410.4" textLength="1134.6" clip-path="url(#terminal-3772557587-line-16)">apache-airflow-providers-apache-pig&#160;|&#160;apache-airflow-providers-apache-pinot&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-3772557587-r4" x="1451 [...]
+</text><text class="terminal-3772557587-r4" x="0" y="434.8" textLength="12.2" clip-path="url(#terminal-3772557587-line-17)">│</text><text class="terminal-3772557587-r7" x="305" y="434.8" textLength="1134.6" clip-path="url(#terminal-3772557587-line-17)">apache-airflow-providers-apache-spark&#160;|&#160;apache-airflow-providers-apache-sqoop&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-3772557587-r4" x="1451.8" y="434 [...]
+</text><text class="terminal-3772557587-r4" x="0" y="459.2" textLength="12.2" clip-path="url(#terminal-3772557587-line-18)">│</text><text class="terminal-3772557587-r7" x="305" y="459.2" textLength="1134.6" clip-path="url(#terminal-3772557587-line-18)">apache-airflow-providers-arangodb&#160;|&#160;apache-airflow-providers-asana&#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><text class="terminal-3772557587-r4" x="0" y="483.6" textLength="12.2" clip-path="url(#terminal-3772557587-line-19)">│</text><text class="terminal-3772557587-r7" x="305" y="483.6" textLength="1134.6" clip-path="url(#terminal-3772557587-line-19)">apache-airflow-providers-celery&#160;|&#160;apache-airflow-providers-cloudant&#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><tex [...]
+</text><text class="terminal-3772557587-r4" x="0" y="508" textLength="12.2" clip-path="url(#terminal-3772557587-line-20)">│</text><text class="terminal-3772557587-r7" x="305" y="508" textLength="1134.6" clip-path="url(#terminal-3772557587-line-20)">apache-airflow-providers-cncf-kubernetes&#160;|&#160;apache-airflow-providers-common-sql&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-3772557587-r4" x="1451.8" y="508" textLen [...]
+</text><text class="terminal-3772557587-r4" x="0" y="532.4" textLength="12.2" clip-path="url(#terminal-3772557587-line-21)">│</text><text class="terminal-3772557587-r7" x="305" y="532.4" textLength="1134.6" clip-path="url(#terminal-3772557587-line-21)">apache-airflow-providers-databricks&#160;|&#160;apache-airflow-providers-datadog&#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-3772557587-r4" x="0" y="556.8" textLength="12.2" clip-path="url(#terminal-3772557587-line-22)">│</text><text class="terminal-3772557587-r7" x="305" y="556.8" textLength="1134.6" clip-path="url(#terminal-3772557587-line-22)">apache-airflow-providers-dbt-cloud&#160;|&#160;apache-airflow-providers-dingding&#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-3772557587-r4" x="0" y="581.2" textLength="12.2" clip-path="url(#terminal-3772557587-line-23)">│</text><text class="terminal-3772557587-r7" x="305" y="581.2" textLength="1134.6" clip-path="url(#terminal-3772557587-line-23)">apache-airflow-providers-discord&#160;|&#160;apache-airflow-providers-docker&#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><text class="terminal-3772557587-r4" x="0" y="605.6" textLength="12.2" clip-path="url(#terminal-3772557587-line-24)">│</text><text class="terminal-3772557587-r7" x="305" y="605.6" textLength="1134.6" clip-path="url(#terminal-3772557587-line-24)">apache-airflow-providers-elasticsearch&#160;|&#160;apache-airflow-providers-exasol&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-3772557 [...]
+</text><text class="terminal-3772557587-r4" x="0" y="630" textLength="12.2" clip-path="url(#terminal-3772557587-line-25)">│</text><text class="terminal-3772557587-r7" x="305" y="630" textLength="1134.6" clip-path="url(#terminal-3772557587-line-25)">apache-airflow-providers-facebook&#160;|&#160;apache-airflow-providers-ftp&#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-3772557587-r4" x="0" y="654.4" textLength="12.2" clip-path="url(#terminal-3772557587-line-26)">│</text><text class="terminal-3772557587-r7" x="305" y="654.4" textLength="1134.6" clip-path="url(#terminal-3772557587-line-26)">apache-airflow-providers-github&#160;|&#160;apache-airflow-providers-google&#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-3772557587-r4" x="0" y="678.8" textLength="12.2" clip-path="url(#terminal-3772557587-line-27)">│</text><text class="terminal-3772557587-r7" x="305" y="678.8" textLength="1134.6" clip-path="url(#terminal-3772557587-line-27)">apache-airflow-providers-grpc&#160;|&#160;apache-airflow-providers-hashicorp&#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><text class="terminal-3772557587-r4" x="0" y="703.2" textLength="12.2" clip-path="url(#terminal-3772557587-line-28)">│</text><text class="terminal-3772557587-r7" x="305" y="703.2" textLength="1134.6" clip-path="url(#terminal-3772557587-line-28)">apache-airflow-providers-http&#160;|&#160;apache-airflow-providers-imap&#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-3772557587-r4" x="0" y="727.6" textLength="12.2" clip-path="url(#terminal-3772557587-line-29)">│</text><text class="terminal-3772557587-r7" x="305" y="727.6" textLength="1134.6" clip-path="url(#terminal-3772557587-line-29)">apache-airflow-providers-influxdb&#160;|&#160;apache-airflow-providers-jdbc&#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-3772557587-r4" x="0" y="752" textLength="12.2" clip-path="url(#terminal-3772557587-line-30)">│</text><text class="terminal-3772557587-r7" x="305" y="752" textLength="1134.6" clip-path="url(#terminal-3772557587-line-30)">apache-airflow-providers-jenkins&#160;|&#160;apache-airflow-providers-jira&#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-3772557587-r4" x="0" y="776.4" textLength="12.2" clip-path="url(#terminal-3772557587-line-31)">│</text><text class="terminal-3772557587-r7" x="305" y="776.4" textLength="1134.6" clip-path="url(#terminal-3772557587-line-31)">apache-airflow-providers-microsoft-azure&#160;|&#160;apache-airflow-providers-microsoft-mssql&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-3772557587-r4" x="1451.8" y="776.4" textLength="12.2" clip-pat [...]
+</text><text class="terminal-3772557587-r4" x="0" y="800.8" textLength="12.2" clip-path="url(#terminal-3772557587-line-32)">│</text><text class="terminal-3772557587-r7" x="305" y="800.8" textLength="1134.6" clip-path="url(#terminal-3772557587-line-32)">apache-airflow-providers-microsoft-psrp&#160;|&#160;apache-airflow-providers-microsoft-winrm&#160;|&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-3772557587-r4" x="1451.8" y="800.8" textLength="12.2" cli [...]
+</text><text class="terminal-3772557587-r4" x="0" y="825.2" textLength="12.2" clip-path="url(#terminal-3772557587-line-33)">│</text><text class="terminal-3772557587-r7" x="305" y="825.2" textLength="1134.6" clip-path="url(#terminal-3772557587-line-33)">apache-airflow-providers-mongo&#160;|&#160;apache-airflow-providers-mysql&#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-3772557587-r4" x="0" y="849.6" textLength="12.2" clip-path="url(#terminal-3772557587-line-34)">│</text><text class="terminal-3772557587-r7" x="305" y="849.6" textLength="1134.6" clip-path="url(#terminal-3772557587-line-34)">apache-airflow-providers-neo4j&#160;|&#160;apache-airflow-providers-odbc&#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-3772557587-r4" x="0" y="874" textLength="12.2" clip-path="url(#terminal-3772557587-line-35)">│</text><text class="terminal-3772557587-r7" x="305" y="874" textLength="1134.6" clip-path="url(#terminal-3772557587-line-35)">apache-airflow-providers-openfaas&#160;|&#160;apache-airflow-providers-opsgenie&#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="termi [...]
+</text><text class="terminal-3772557587-r4" x="0" y="898.4" textLength="12.2" clip-path="url(#terminal-3772557587-line-36)">│</text><text class="terminal-3772557587-r7" x="305" y="898.4" textLength="1134.6" clip-path="url(#terminal-3772557587-line-36)">apache-airflow-providers-oracle&#160;|&#160;apache-airflow-providers-pagerduty&#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-3772557587-r4" x="0" y="922.8" textLength="12.2" clip-path="url(#terminal-3772557587-line-37)">│</text><text class="terminal-3772557587-r7" x="305" y="922.8" textLength="1134.6" clip-path="url(#terminal-3772557587-line-37)">apache-airflow-providers-papermill&#160;|&#160;apache-airflow-providers-plexus&#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-3772557587-r4" x="0" y="947.2" textLength="12.2" clip-path="url(#terminal-3772557587-line-38)">│</text><text class="terminal-3772557587-r7" x="305" y="947.2" textLength="1134.6" clip-path="url(#terminal-3772557587-line-38)">apache-airflow-providers-postgres&#160;|&#160;apache-airflow-providers-presto&#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><tex [...]
+</text><text class="terminal-3772557587-r4" x="0" y="971.6" textLength="12.2" clip-path="url(#terminal-3772557587-line-39)">│</text><text class="terminal-3772557587-r7" x="305" y="971.6" textLength="1134.6" clip-path="url(#terminal-3772557587-line-39)">apache-airflow-providers-qubole&#160;|&#160;apache-airflow-providers-redis&#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-3772557587-r4" x="0" y="996" textLength="12.2" clip-path="url(#terminal-3772557587-line-40)">│</text><text class="terminal-3772557587-r7" x="305" y="996" textLength="1134.6" clip-path="url(#terminal-3772557587-line-40)">apache-airflow-providers-salesforce&#160;|&#160;apache-airflow-providers-samba&#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=" [...]
+</text><text class="terminal-3772557587-r4" x="0" y="1020.4" textLength="12.2" clip-path="url(#terminal-3772557587-line-41)">│</text><text class="terminal-3772557587-r7" x="305" y="1020.4" textLength="1134.6" clip-path="url(#terminal-3772557587-line-41)">apache-airflow-providers-segment&#160;|&#160;apache-airflow-providers-sendgrid&#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 c [...]
+</text><text class="terminal-3772557587-r4" x="0" y="1044.8" textLength="12.2" clip-path="url(#terminal-3772557587-line-42)">│</text><text class="terminal-3772557587-r7" x="305" y="1044.8" textLength="1134.6" clip-path="url(#terminal-3772557587-line-42)">apache-airflow-providers-sftp&#160;|&#160;apache-airflow-providers-singularity&#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 c [...]
+</text><text class="terminal-3772557587-r4" x="0" y="1069.2" textLength="12.2" clip-path="url(#terminal-3772557587-line-43)">│</text><text class="terminal-3772557587-r7" x="305" y="1069.2" textLength="1134.6" clip-path="url(#terminal-3772557587-line-43)">apache-airflow-providers-slack&#160;|&#160;apache-airflow-providers-snowflake&#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><t [...]
+</text><text class="terminal-3772557587-r4" x="0" y="1093.6" textLength="12.2" clip-path="url(#terminal-3772557587-line-44)">│</text><text class="terminal-3772557587-r7" x="305" y="1093.6" textLength="1134.6" clip-path="url(#terminal-3772557587-line-44)">apache-airflow-providers-sqlite&#160;|&#160;apache-airflow-providers-ssh&#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-3772557587-r4" x="0" y="1118" textLength="12.2" clip-path="url(#terminal-3772557587-line-45)">│</text><text class="terminal-3772557587-r7" x="305" y="1118" textLength="1134.6" clip-path="url(#terminal-3772557587-line-45)">apache-airflow-providers-tableau&#160;|&#160;apache-airflow-providers-tabular&#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  [...]
+</text><text class="terminal-3772557587-r4" x="0" y="1142.4" textLength="12.2" clip-path="url(#terminal-3772557587-line-46)">│</text><text class="terminal-3772557587-r7" x="305" y="1142.4" textLength="1134.6" clip-path="url(#terminal-3772557587-line-46)">apache-airflow-providers-telegram&#160;|&#160;apache-airflow-providers-trino&#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;</te [...]
+</text><text class="terminal-3772557587-r4" x="0" y="1166.8" textLength="12.2" clip-path="url(#terminal-3772557587-line-47)">│</text><text class="terminal-3772557587-r7" x="305" y="1166.8" textLength="1134.6" clip-path="url(#terminal-3772557587-line-47)">apache-airflow-providers-vertica&#160;|&#160;apache-airflow-providers-yandex&#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;</te [...]
+</text><text class="terminal-3772557587-r4" x="0" y="1191.2" textLength="12.2" clip-path="url(#terminal-3772557587-line-48)">│</text><text class="terminal-3772557587-r7" x="305" y="1191.2" textLength="1134.6" clip-path="url(#terminal-3772557587-line-48)">apache-airflow-providers-zendesk&#160;|&#160;docker-stack&#160;|&#160;helm-chart)&#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-3772557587-r4" x="0" y="1215.6" textLength="1464" clip-path="url(#terminal-3772557587-line-49)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-3772557587-r2" x="1464" y="1215.6" textLength="12.2" clip-path="url(#terminal-3772557587-line-49)">
+</text><text class="terminal-3772557587-r4" x="0" y="1240" textLength="24.4" clip-path="url(#terminal-3772557587-line-50)">╭─</text><text class="terminal-3772557587-r4" x="24.4" y="1240" textLength="1415.2" clip-path="url(#terminal-3772557587-line-50)">&#160;Options&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-3772557587-r4" x="1439.6" y="1240" textLength="24.4" clip-path="url(#terminal-377255 [...]
+</text><text class="terminal-3772557587-r4" x="0" y="1264.4" textLength="12.2" clip-path="url(#terminal-3772557587-line-51)">│</text><text class="terminal-3772557587-r5" x="24.4" y="1264.4" textLength="12.2" clip-path="url(#terminal-3772557587-line-51)">-</text><text class="terminal-3772557587-r5" x="36.6" y="1264.4" textLength="97.6" clip-path="url(#terminal-3772557587-line-51)">-verbose</text><text class="terminal-3772557587-r6" x="280.6" y="1264.4" textLength="24.4" clip-path="url(#te [...]
+</text><text class="terminal-3772557587-r4" x="0" y="1288.8" textLength="12.2" clip-path="url(#terminal-3772557587-line-52)">│</text><text class="terminal-3772557587-r5" x="24.4" y="1288.8" textLength="12.2" clip-path="url(#terminal-3772557587-line-52)">-</text><text class="terminal-3772557587-r5" x="36.6" y="1288.8" textLength="48.8" clip-path="url(#terminal-3772557587-line-52)">-dry</text><text class="terminal-3772557587-r5" x="85.4" y="1288.8" textLength="48.8" clip-path="url(#termina [...]
+</text><text class="terminal-3772557587-r4" x="0" y="1313.2" textLength="12.2" clip-path="url(#terminal-3772557587-line-53)">│</text><text class="terminal-3772557587-r5" x="24.4" y="1313.2" textLength="12.2" clip-path="url(#terminal-3772557587-line-53)">-</text><text class="terminal-3772557587-r5" x="36.6" y="1313.2" textLength="85.4" clip-path="url(#terminal-3772557587-line-53)">-github</text><text class="terminal-3772557587-r5" x="122" y="1313.2" textLength="134.2" clip-path="url(#term [...]
+</text><text class="terminal-3772557587-r4" x="0" y="1337.6" textLength="12.2" clip-path="url(#terminal-3772557587-line-54)">│</text><text class="terminal-3772557587-r5" x="24.4" y="1337.6" textLength="12.2" clip-path="url(#terminal-3772557587-line-54)">-</text><text class="terminal-3772557587-r5" x="36.6" y="1337.6" textLength="61" clip-path="url(#terminal-3772557587-line-54)">-help</text><text class="terminal-3772557587-r6" x="280.6" y="1337.6" textLength="24.4" clip-path="url(#termina [...]
+</text><text class="terminal-3772557587-r4" x="0" y="1362" textLength="1464" clip-path="url(#terminal-3772557587-line-55)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-3772557587-r2" x="1464" y="1362" textLength="12.2" clip-path="url(#terminal-3772557587-line-55)">
 </text>
     </g>
     </g>
diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt
index df72bfd829..2dd07945e5 100644
--- a/images/breeze/output-commands-hash.txt
+++ b/images/breeze/output-commands-hash.txt
@@ -3,7 +3,7 @@
 # Please do not solve it but run `breeze regenerate-command-images`.
 # This command should fix the conflict and regenerate help images that you have conflict with.
 main:fa4319079b275ce966502346f083f2e3
-build-docs:39e674e357429ab1be7cc363cb793434
+build-docs:5ac8cf0870ec66fc7ebcf2363e823178
 build-image:b62509a59badf3aa230e4562df751002
 build-prod-image:1902ec077a6d70336de6038d13472ef3
 cleanup:9a94bd1063296ea86e895f671db0b330
@@ -16,8 +16,8 @@ fix-ownership:596143cc74217f0a90850a554220ea45
 free-space:bb8e7ac63d12ab3ede272a898de2f527
 generate-constraints:a5120e79439f30eb7fbee929dca23156
 prepare-airflow-package:cff9d88ca313db10f3cc464c6798f6be
-prepare-provider-documentation:95c864f8a656a95cac7d9c682cb75773
-prepare-provider-packages:33c0fe04ad4c6068b69ad1361b142057
+prepare-provider-documentation:ff90e2d37c629e0f7b1f5e8bc723d9db
+prepare-provider-packages:349292885c763f32db2bb8f99ae0ae59
 pull-image:a9bb83372b5da5212f48e2affeedc551
 pull-prod-image:6e8467a2b8c833a392c8bdd65189363e
 regenerate-command-images:4fd2e7ecbfd6eebb18b854f3eb0f29c8
diff --git a/images/breeze/output-prepare-provider-documentation.svg b/images/breeze/output-prepare-provider-documentation.svg
index 6390fc20d9..c413c49f8c 100644
--- a/images/breeze/output-prepare-provider-documentation.svg
+++ b/images/breeze/output-prepare-provider-documentation.svg
@@ -19,149 +19,149 @@
         font-weight: 700;
     }
 
-    .terminal-1998463419-matrix {
+    .terminal-1019290203-matrix {
         font-family: Fira Code, monospace;
         font-size: 20px;
         line-height: 24.4px;
         font-variant-east-asian: full-width;
     }
 
-    .terminal-1998463419-title {
+    .terminal-1019290203-title {
         font-size: 18px;
         font-weight: bold;
         font-family: arial;
     }
 
-    .terminal-1998463419-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-1998463419-r2 { fill: #c5c8c6 }
-.terminal-1998463419-r3 { fill: #d0b344;font-weight: bold }
-.terminal-1998463419-r4 { fill: #868887 }
-.terminal-1998463419-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-1998463419-r6 { fill: #98a84b;font-weight: bold }
-.terminal-1998463419-r7 { fill: #8d7b39 }
+    .terminal-1019290203-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-1019290203-r2 { fill: #c5c8c6 }
+.terminal-1019290203-r3 { fill: #d0b344;font-weight: bold }
+.terminal-1019290203-r4 { fill: #868887 }
+.terminal-1019290203-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-1019290203-r6 { fill: #98a84b;font-weight: bold }
+.terminal-1019290203-r7 { fill: #8d7b39 }
     </style>
 
     <defs>
-    <clipPath id="terminal-1998463419-clip-terminal">
+    <clipPath id="terminal-1019290203-clip-terminal">
       <rect x="0" y="0" width="1463.0" height="657.8" />
     </clipPath>
-    <clipPath id="terminal-1998463419-line-0">
+    <clipPath id="terminal-1019290203-line-0">
     <rect x="0" y="1.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-1">
+<clipPath id="terminal-1019290203-line-1">
     <rect x="0" y="25.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-2">
+<clipPath id="terminal-1019290203-line-2">
     <rect x="0" y="50.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-3">
+<clipPath id="terminal-1019290203-line-3">
     <rect x="0" y="74.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-4">
+<clipPath id="terminal-1019290203-line-4">
     <rect x="0" y="99.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-5">
+<clipPath id="terminal-1019290203-line-5">
     <rect x="0" y="123.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-6">
+<clipPath id="terminal-1019290203-line-6">
     <rect x="0" y="147.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-7">
+<clipPath id="terminal-1019290203-line-7">
     <rect x="0" y="172.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-8">
+<clipPath id="terminal-1019290203-line-8">
     <rect x="0" y="196.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-9">
+<clipPath id="terminal-1019290203-line-9">
     <rect x="0" y="221.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-10">
+<clipPath id="terminal-1019290203-line-10">
     <rect x="0" y="245.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-11">
+<clipPath id="terminal-1019290203-line-11">
     <rect x="0" y="269.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-12">
+<clipPath id="terminal-1019290203-line-12">
     <rect x="0" y="294.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-13">
+<clipPath id="terminal-1019290203-line-13">
     <rect x="0" y="318.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-14">
+<clipPath id="terminal-1019290203-line-14">
     <rect x="0" y="343.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-15">
+<clipPath id="terminal-1019290203-line-15">
     <rect x="0" y="367.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-16">
+<clipPath id="terminal-1019290203-line-16">
     <rect x="0" y="391.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-17">
+<clipPath id="terminal-1019290203-line-17">
     <rect x="0" y="416.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-18">
+<clipPath id="terminal-1019290203-line-18">
     <rect x="0" y="440.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-19">
+<clipPath id="terminal-1019290203-line-19">
     <rect x="0" y="465.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-20">
+<clipPath id="terminal-1019290203-line-20">
     <rect x="0" y="489.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-21">
+<clipPath id="terminal-1019290203-line-21">
     <rect x="0" y="513.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-22">
+<clipPath id="terminal-1019290203-line-22">
     <rect x="0" y="538.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-23">
+<clipPath id="terminal-1019290203-line-23">
     <rect x="0" y="562.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-24">
+<clipPath id="terminal-1019290203-line-24">
     <rect x="0" y="587.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1998463419-line-25">
+<clipPath id="terminal-1019290203-line-25">
     <rect x="0" y="611.5" 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="706.8" rx="8"/><text class="terminal-1998463419-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;prepare-provider-documentation</text>
+    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="706.8" rx="8"/><text class="terminal-1019290203-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;prepare-provider-documentation</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-1998463419-clip-terminal)">
+    <g transform="translate(9, 41)" clip-path="url(#terminal-1019290203-clip-terminal)">
     
-    <g class="terminal-1998463419-matrix">
-    <text class="terminal-1998463419-r2" x="1464" y="20" textLength="12.2" clip-path="url(#terminal-1998463419-line-0)">
-</text><text class="terminal-1998463419-r3" x="12.2" y="44.4" textLength="85.4" clip-path="url(#terminal-1998463419-line-1)">Usage:&#160;</text><text class="terminal-1998463419-r1" x="97.6" y="44.4" textLength="1342" clip-path="url(#terminal-1998463419-line-1)">breeze&#160;prepare-provider-documentation&#160;[OPTIONS]&#160;[airbyte&#160;|&#160;alibaba&#160;|&#160;amazon&#160;|&#160;apache.beam&#160;|&#160;apache.cassandra&#160;|</text><text class="terminal-1998463419-r2" x="1464" y="44.4 [...]
-</text><text class="terminal-1998463419-r1" x="12.2" y="68.8" textLength="1427.4" clip-path="url(#terminal-1998463419-line-2)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;apache.drill&#160;|&#160;apache.druid&#160;|&#160;apache.hdfs&#160;|&#160;apache.hive&#160;|&#1 [...]
-</text><text class="terminal-1998463419-r1" x="12.2" y="93.2" textLength="1415.2" clip-path="url(#terminal-1998463419-line-3)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;apache.livy&#160;|&#160;apache.pig&#160;|&#160;apache.pinot&#160;|&#160;apache.spark&#160;|&#16 [...]
-</text><text class="terminal-1998463419-r1" x="12.2" y="117.6" textLength="1366.4" clip-path="url(#terminal-1998463419-line-4)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;arangodb&#160;|&#160;asana&#160;|&#160;celery&#160;|&#160;cloudant&#160;|&#160;cncf.kubernetes [...]
-</text><text class="terminal-1998463419-r1" x="12.2" y="142" textLength="1329.8" clip-path="url(#terminal-1998463419-line-5)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;databricks&#160;|&#160;datadog&#160;|&#160;dbt.cloud&#160;|&#160;dingding&#160;|&#160;discord&#1 [...]
-</text><text class="terminal-1998463419-r1" x="12.2" y="166.4" textLength="1354.2" clip-path="url(#terminal-1998463419-line-6)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;elasticsearch&#160;|&#160;exasol&#160;|&#160;facebook&#160;|&#160;ftp&#160;|&#160;github&#160; [...]
-</text><text class="terminal-1998463419-r1" x="12.2" y="190.8" textLength="1281" clip-path="url(#terminal-1998463419-line-7)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;hashicorp&#160;|&#160;http&#160;|&#160;imap&#160;|&#160;influxdb&#160;|&#160;jdbc&#160;|&#160;je [...]
-</text><text class="terminal-1998463419-r1" x="12.2" y="215.2" textLength="1403" clip-path="url(#terminal-1998463419-line-8)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;microsoft.azure&#160;|&#160;microsoft.mssql&#160;|&#160;microsoft.psrp&#160;|&#160;microsoft.win [...]
-</text><text class="terminal-1998463419-r1" x="12.2" y="239.6" textLength="1439.6" clip-path="url(#terminal-1998463419-line-9)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;mongo&#160;|&#160;mysql&#160;|&#160;neo4j&#160;|&#160;odbc&#160;|&#160;openfaas&#160;|&#160;op [...]
-</text><text class="terminal-1998463419-r1" x="12.2" y="264" textLength="1403" clip-path="url(#terminal-1998463419-line-10)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;papermill&#160;|&#160;plexus&#160;|&#160;postgres&#160;|&#160;presto&#160;|&#160;qubole&#160;|&#1 [...]
-</text><text class="terminal-1998463419-r1" x="12.2" y="288.4" textLength="1390.8" clip-path="url(#terminal-1998463419-line-11)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;samba&#160;|&#160;segment&#160;|&#160;sendgrid&#160;|&#160;sftp&#160;|&#160;singularity&#160; [...]
-</text><text class="terminal-1998463419-r1" x="12.2" y="312.8" textLength="1427.4" clip-path="url(#terminal-1998463419-line-12)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;sqlite&#160;|&#160;ssh&#160;|&#160;tableau&#160;|&#160;tabular&#160;|&#160;telegram&#160;|&#1 [...]
-</text><text class="terminal-1998463419-r1" x="12.2" y="337.2" textLength="683.2" clip-path="url(#terminal-1998463419-line-13)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;zendesk]...</text><text class="terminal-1998463419-r2" x="1464" y="337.2" textLength="12.2" cl [...]
-</text><text class="terminal-1998463419-r2" x="1464" y="361.6" textLength="12.2" clip-path="url(#terminal-1998463419-line-14)">
-</text><text class="terminal-1998463419-r2" x="12.2" y="386" textLength="780.8" clip-path="url(#terminal-1998463419-line-15)">Prepare&#160;CHANGELOG,&#160;README&#160;and&#160;COMMITS&#160;information&#160;for&#160;providers.</text><text class="terminal-1998463419-r2" x="1464" y="386" textLength="12.2" clip-path="url(#terminal-1998463419-line-15)">
-</text><text class="terminal-1998463419-r2" x="1464" y="410.4" textLength="12.2" clip-path="url(#terminal-1998463419-line-16)">
-</text><text class="terminal-1998463419-r4" x="0" y="434.8" textLength="24.4" clip-path="url(#terminal-1998463419-line-17)">╭─</text><text class="terminal-1998463419-r4" x="24.4" y="434.8" textLength="1415.2" clip-path="url(#terminal-1998463419-line-17)">&#160;Provider&#160;documentation&#160;preparation&#160;flags&#160;──────────────────────────────────────────────────────────────────────────</text><text class="terminal-1998463419-r4" x="1439.6" y="434.8" textLength="24.4" clip-path="ur [...]
-</text><text class="terminal-1998463419-r4" x="0" y="459.2" textLength="12.2" clip-path="url(#terminal-1998463419-line-18)">│</text><text class="terminal-1998463419-r5" x="24.4" y="459.2" textLength="12.2" clip-path="url(#terminal-1998463419-line-18)">-</text><text class="terminal-1998463419-r5" x="36.6" y="459.2" textLength="73.2" clip-path="url(#terminal-1998463419-line-18)">-debug</text><text class="terminal-1998463419-r2" x="158.6" y="459.2" textLength="878.4" clip-path="url(#termina [...]
-</text><text class="terminal-1998463419-r4" x="0" y="483.6" textLength="1464" clip-path="url(#terminal-1998463419-line-19)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1998463419-r2" x="1464" y="483.6" textLength="12.2" clip-path="url(#terminal-1998463419-line-19)">
-</text><text class="terminal-1998463419-r4" x="0" y="508" textLength="24.4" clip-path="url(#terminal-1998463419-line-20)">╭─</text><text class="terminal-1998463419-r4" x="24.4" y="508" textLength="1415.2" clip-path="url(#terminal-1998463419-line-20)">&#160;Options&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-1998463419-r4" x="1439.6" y="508" textLength="24.4" clip-path="url(#terminal-199846341 [...]
-</text><text class="terminal-1998463419-r4" x="0" y="532.4" textLength="12.2" clip-path="url(#terminal-1998463419-line-21)">│</text><text class="terminal-1998463419-r5" x="24.4" y="532.4" textLength="12.2" clip-path="url(#terminal-1998463419-line-21)">-</text><text class="terminal-1998463419-r5" x="36.6" y="532.4" textLength="97.6" clip-path="url(#terminal-1998463419-line-21)">-verbose</text><text class="terminal-1998463419-r6" x="280.6" y="532.4" textLength="24.4" clip-path="url(#termin [...]
-</text><text class="terminal-1998463419-r4" x="0" y="556.8" textLength="12.2" clip-path="url(#terminal-1998463419-line-22)">│</text><text class="terminal-1998463419-r5" x="24.4" y="556.8" textLength="12.2" clip-path="url(#terminal-1998463419-line-22)">-</text><text class="terminal-1998463419-r5" x="36.6" y="556.8" textLength="48.8" clip-path="url(#terminal-1998463419-line-22)">-dry</text><text class="terminal-1998463419-r5" x="85.4" y="556.8" textLength="48.8" clip-path="url(#terminal-19 [...]
-</text><text class="terminal-1998463419-r4" x="0" y="581.2" textLength="12.2" clip-path="url(#terminal-1998463419-line-23)">│</text><text class="terminal-1998463419-r5" x="24.4" y="581.2" textLength="12.2" clip-path="url(#terminal-1998463419-line-23)">-</text><text class="terminal-1998463419-r5" x="36.6" y="581.2" textLength="85.4" clip-path="url(#terminal-1998463419-line-23)">-github</text><text class="terminal-1998463419-r5" x="122" y="581.2" textLength="134.2" clip-path="url(#terminal [...]
-</text><text class="terminal-1998463419-r4" x="0" y="605.6" textLength="12.2" clip-path="url(#terminal-1998463419-line-24)">│</text><text class="terminal-1998463419-r5" x="24.4" y="605.6" textLength="12.2" clip-path="url(#terminal-1998463419-line-24)">-</text><text class="terminal-1998463419-r5" x="36.6" y="605.6" textLength="85.4" clip-path="url(#terminal-1998463419-line-24)">-answer</text><text class="terminal-1998463419-r6" x="280.6" y="605.6" textLength="24.4" clip-path="url(#termina [...]
-</text><text class="terminal-1998463419-r4" x="0" y="630" textLength="12.2" clip-path="url(#terminal-1998463419-line-25)">│</text><text class="terminal-1998463419-r5" x="24.4" y="630" textLength="12.2" clip-path="url(#terminal-1998463419-line-25)">-</text><text class="terminal-1998463419-r5" x="36.6" y="630" textLength="61" clip-path="url(#terminal-1998463419-line-25)">-help</text><text class="terminal-1998463419-r6" x="280.6" y="630" textLength="24.4" clip-path="url(#terminal-1998463419 [...]
-</text><text class="terminal-1998463419-r4" x="0" y="654.4" textLength="1464" clip-path="url(#terminal-1998463419-line-26)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1998463419-r2" x="1464" y="654.4" textLength="12.2" clip-path="url(#terminal-1998463419-line-26)">
+    <g class="terminal-1019290203-matrix">
+    <text class="terminal-1019290203-r2" x="1464" y="20" textLength="12.2" clip-path="url(#terminal-1019290203-line-0)">
+</text><text class="terminal-1019290203-r3" x="12.2" y="44.4" textLength="85.4" clip-path="url(#terminal-1019290203-line-1)">Usage:&#160;</text><text class="terminal-1019290203-r1" x="97.6" y="44.4" textLength="1342" clip-path="url(#terminal-1019290203-line-1)">breeze&#160;prepare-provider-documentation&#160;[OPTIONS]&#160;[airbyte&#160;|&#160;alibaba&#160;|&#160;amazon&#160;|&#160;apache.beam&#160;|&#160;apache.cassandra&#160;|</text><text class="terminal-1019290203-r2" x="1464" y="44.4 [...]
+</text><text class="terminal-1019290203-r1" x="12.2" y="68.8" textLength="1427.4" clip-path="url(#terminal-1019290203-line-2)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;apache.drill&#160;|&#160;apache.druid&#160;|&#160;apache.hdfs&#160;|&#160;apache.hive&#160;|&#1 [...]
+</text><text class="terminal-1019290203-r1" x="12.2" y="93.2" textLength="1415.2" clip-path="url(#terminal-1019290203-line-3)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;apache.livy&#160;|&#160;apache.pig&#160;|&#160;apache.pinot&#160;|&#160;apache.spark&#160;|&#16 [...]
+</text><text class="terminal-1019290203-r1" x="12.2" y="117.6" textLength="1390.8" clip-path="url(#terminal-1019290203-line-4)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;arangodb&#160;|&#160;asana&#160;|&#160;celery&#160;|&#160;cloudant&#160;|&#160;cncf.kubernetes [...]
+</text><text class="terminal-1019290203-r1" x="12.2" y="142" textLength="1329.8" clip-path="url(#terminal-1019290203-line-5)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;databricks&#160;|&#160;datadog&#160;|&#160;dbt.cloud&#160;|&#160;dingding&#160;|&#160;discord&#1 [...]
+</text><text class="terminal-1019290203-r1" x="12.2" y="166.4" textLength="1354.2" clip-path="url(#terminal-1019290203-line-6)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;elasticsearch&#160;|&#160;exasol&#160;|&#160;facebook&#160;|&#160;ftp&#160;|&#160;github&#160; [...]
+</text><text class="terminal-1019290203-r1" x="12.2" y="190.8" textLength="1281" clip-path="url(#terminal-1019290203-line-7)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;hashicorp&#160;|&#160;http&#160;|&#160;imap&#160;|&#160;influxdb&#160;|&#160;jdbc&#160;|&#160;je [...]
+</text><text class="terminal-1019290203-r1" x="12.2" y="215.2" textLength="1403" clip-path="url(#terminal-1019290203-line-8)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;microsoft.azure&#160;|&#160;microsoft.mssql&#160;|&#160;microsoft.psrp&#160;|&#160;microsoft.win [...]
+</text><text class="terminal-1019290203-r1" x="12.2" y="239.6" textLength="1439.6" clip-path="url(#terminal-1019290203-line-9)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;mongo&#160;|&#160;mysql&#160;|&#160;neo4j&#160;|&#160;odbc&#160;|&#160;openfaas&#160;|&#160;op [...]
+</text><text class="terminal-1019290203-r1" x="12.2" y="264" textLength="1403" clip-path="url(#terminal-1019290203-line-10)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;papermill&#160;|&#160;plexus&#160;|&#160;postgres&#160;|&#160;presto&#160;|&#160;qubole&#160;|&#1 [...]
+</text><text class="terminal-1019290203-r1" x="12.2" y="288.4" textLength="1390.8" clip-path="url(#terminal-1019290203-line-11)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;samba&#160;|&#160;segment&#160;|&#160;sendgrid&#160;|&#160;sftp&#160;|&#160;singularity&#160; [...]
+</text><text class="terminal-1019290203-r1" x="12.2" y="312.8" textLength="1427.4" clip-path="url(#terminal-1019290203-line-12)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;sqlite&#160;|&#160;ssh&#160;|&#160;tableau&#160;|&#160;tabular&#160;|&#160;telegram&#160;|&#1 [...]
+</text><text class="terminal-1019290203-r1" x="12.2" y="337.2" textLength="683.2" clip-path="url(#terminal-1019290203-line-13)">&#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;&#160;&#160;&#160;&#160;&#160;&#160;zendesk]...</text><text class="terminal-1019290203-r2" x="1464" y="337.2" textLength="12.2" cl [...]
+</text><text class="terminal-1019290203-r2" x="1464" y="361.6" textLength="12.2" clip-path="url(#terminal-1019290203-line-14)">
+</text><text class="terminal-1019290203-r2" x="12.2" y="386" textLength="780.8" clip-path="url(#terminal-1019290203-line-15)">Prepare&#160;CHANGELOG,&#160;README&#160;and&#160;COMMITS&#160;information&#160;for&#160;providers.</text><text class="terminal-1019290203-r2" x="1464" y="386" textLength="12.2" clip-path="url(#terminal-1019290203-line-15)">
+</text><text class="terminal-1019290203-r2" x="1464" y="410.4" textLength="12.2" clip-path="url(#terminal-1019290203-line-16)">
+</text><text class="terminal-1019290203-r4" x="0" y="434.8" textLength="24.4" clip-path="url(#terminal-1019290203-line-17)">╭─</text><text class="terminal-1019290203-r4" x="24.4" y="434.8" textLength="1415.2" clip-path="url(#terminal-1019290203-line-17)">&#160;Provider&#160;documentation&#160;preparation&#160;flags&#160;──────────────────────────────────────────────────────────────────────────</text><text class="terminal-1019290203-r4" x="1439.6" y="434.8" textLength="24.4" clip-path="ur [...]
+</text><text class="terminal-1019290203-r4" x="0" y="459.2" textLength="12.2" clip-path="url(#terminal-1019290203-line-18)">│</text><text class="terminal-1019290203-r5" x="24.4" y="459.2" textLength="12.2" clip-path="url(#terminal-1019290203-line-18)">-</text><text class="terminal-1019290203-r5" x="36.6" y="459.2" textLength="73.2" clip-path="url(#terminal-1019290203-line-18)">-debug</text><text class="terminal-1019290203-r2" x="158.6" y="459.2" textLength="878.4" clip-path="url(#termina [...]
+</text><text class="terminal-1019290203-r4" x="0" y="483.6" textLength="1464" clip-path="url(#terminal-1019290203-line-19)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1019290203-r2" x="1464" y="483.6" textLength="12.2" clip-path="url(#terminal-1019290203-line-19)">
+</text><text class="terminal-1019290203-r4" x="0" y="508" textLength="24.4" clip-path="url(#terminal-1019290203-line-20)">╭─</text><text class="terminal-1019290203-r4" x="24.4" y="508" textLength="1415.2" clip-path="url(#terminal-1019290203-line-20)">&#160;Options&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-1019290203-r4" x="1439.6" y="508" textLength="24.4" clip-path="url(#terminal-101929020 [...]
+</text><text class="terminal-1019290203-r4" x="0" y="532.4" textLength="12.2" clip-path="url(#terminal-1019290203-line-21)">│</text><text class="terminal-1019290203-r5" x="24.4" y="532.4" textLength="12.2" clip-path="url(#terminal-1019290203-line-21)">-</text><text class="terminal-1019290203-r5" x="36.6" y="532.4" textLength="97.6" clip-path="url(#terminal-1019290203-line-21)">-verbose</text><text class="terminal-1019290203-r6" x="280.6" y="532.4" textLength="24.4" clip-path="url(#termin [...]
+</text><text class="terminal-1019290203-r4" x="0" y="556.8" textLength="12.2" clip-path="url(#terminal-1019290203-line-22)">│</text><text class="terminal-1019290203-r5" x="24.4" y="556.8" textLength="12.2" clip-path="url(#terminal-1019290203-line-22)">-</text><text class="terminal-1019290203-r5" x="36.6" y="556.8" textLength="48.8" clip-path="url(#terminal-1019290203-line-22)">-dry</text><text class="terminal-1019290203-r5" x="85.4" y="556.8" textLength="48.8" clip-path="url(#terminal-10 [...]
+</text><text class="terminal-1019290203-r4" x="0" y="581.2" textLength="12.2" clip-path="url(#terminal-1019290203-line-23)">│</text><text class="terminal-1019290203-r5" x="24.4" y="581.2" textLength="12.2" clip-path="url(#terminal-1019290203-line-23)">-</text><text class="terminal-1019290203-r5" x="36.6" y="581.2" textLength="85.4" clip-path="url(#terminal-1019290203-line-23)">-github</text><text class="terminal-1019290203-r5" x="122" y="581.2" textLength="134.2" clip-path="url(#terminal [...]
+</text><text class="terminal-1019290203-r4" x="0" y="605.6" textLength="12.2" clip-path="url(#terminal-1019290203-line-24)">│</text><text class="terminal-1019290203-r5" x="24.4" y="605.6" textLength="12.2" clip-path="url(#terminal-1019290203-line-24)">-</text><text class="terminal-1019290203-r5" x="36.6" y="605.6" textLength="85.4" clip-path="url(#terminal-1019290203-line-24)">-answer</text><text class="terminal-1019290203-r6" x="280.6" y="605.6" textLength="24.4" clip-path="url(#termina [...]
+</text><text class="terminal-1019290203-r4" x="0" y="630" textLength="12.2" clip-path="url(#terminal-1019290203-line-25)">│</text><text class="terminal-1019290203-r5" x="24.4" y="630" textLength="12.2" clip-path="url(#terminal-1019290203-line-25)">-</text><text class="terminal-1019290203-r5" x="36.6" y="630" textLength="61" clip-path="url(#terminal-1019290203-line-25)">-help</text><text class="terminal-1019290203-r6" x="280.6" y="630" textLength="24.4" clip-path="url(#terminal-1019290203 [...]
+</text><text class="terminal-1019290203-r4" x="0" y="654.4" textLength="1464" clip-path="url(#terminal-1019290203-line-26)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1019290203-r2" x="1464" y="654.4" textLength="12.2" clip-path="url(#terminal-1019290203-line-26)">
 </text>
     </g>
     </g>
diff --git a/images/breeze/output-prepare-provider-packages.svg b/images/breeze/output-prepare-provider-packages.svg
index fe0c1733cb..aa4ba89f53 100644
--- a/images/breeze/output-prepare-provider-packages.svg
+++ b/images/breeze/output-prepare-provider-packages.svg
@@ -19,153 +19,153 @@
         font-weight: 700;
     }
 
-    .terminal-3451094975-matrix {
+    .terminal-2104395871-matrix {
         font-family: Fira Code, monospace;
         font-size: 20px;
         line-height: 24.4px;
         font-variant-east-asian: full-width;
     }
 
-    .terminal-3451094975-title {
+    .terminal-2104395871-title {
         font-size: 18px;
         font-weight: bold;
         font-family: arial;
     }
 
-    .terminal-3451094975-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-3451094975-r2 { fill: #c5c8c6 }
-.terminal-3451094975-r3 { fill: #d0b344;font-weight: bold }
-.terminal-3451094975-r4 { fill: #868887 }
-.terminal-3451094975-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-3451094975-r6 { fill: #8d7b39 }
-.terminal-3451094975-r7 { fill: #98a84b;font-weight: bold }
+    .terminal-2104395871-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-2104395871-r2 { fill: #c5c8c6 }
+.terminal-2104395871-r3 { fill: #d0b344;font-weight: bold }
+.terminal-2104395871-r4 { fill: #868887 }
+.terminal-2104395871-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-2104395871-r6 { fill: #8d7b39 }
+.terminal-2104395871-r7 { fill: #98a84b;font-weight: bold }
     </style>
 
     <defs>
-    <clipPath id="terminal-3451094975-clip-terminal">
+    <clipPath id="terminal-2104395871-clip-terminal">
       <rect x="0" y="0" width="1463.0" height="682.1999999999999" />
     </clipPath>
-    <clipPath id="terminal-3451094975-line-0">
+    <clipPath id="terminal-2104395871-line-0">
     <rect x="0" y="1.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-1">
+<clipPath id="terminal-2104395871-line-1">
     <rect x="0" y="25.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-2">
+<clipPath id="terminal-2104395871-line-2">
     <rect x="0" y="50.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-3">
+<clipPath id="terminal-2104395871-line-3">
     <rect x="0" y="74.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-4">
+<clipPath id="terminal-2104395871-line-4">
     <rect x="0" y="99.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-5">
+<clipPath id="terminal-2104395871-line-5">
     <rect x="0" y="123.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-6">
+<clipPath id="terminal-2104395871-line-6">
     <rect x="0" y="147.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-7">
+<clipPath id="terminal-2104395871-line-7">
     <rect x="0" y="172.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-8">
+<clipPath id="terminal-2104395871-line-8">
     <rect x="0" y="196.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-9">
+<clipPath id="terminal-2104395871-line-9">
     <rect x="0" y="221.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-10">
+<clipPath id="terminal-2104395871-line-10">
     <rect x="0" y="245.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-11">
+<clipPath id="terminal-2104395871-line-11">
     <rect x="0" y="269.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-12">
+<clipPath id="terminal-2104395871-line-12">
     <rect x="0" y="294.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-13">
+<clipPath id="terminal-2104395871-line-13">
     <rect x="0" y="318.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-14">
+<clipPath id="terminal-2104395871-line-14">
     <rect x="0" y="343.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-15">
+<clipPath id="terminal-2104395871-line-15">
     <rect x="0" y="367.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-16">
+<clipPath id="terminal-2104395871-line-16">
     <rect x="0" y="391.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-17">
+<clipPath id="terminal-2104395871-line-17">
     <rect x="0" y="416.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-18">
+<clipPath id="terminal-2104395871-line-18">
     <rect x="0" y="440.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-19">
+<clipPath id="terminal-2104395871-line-19">
     <rect x="0" y="465.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-20">
+<clipPath id="terminal-2104395871-line-20">
     <rect x="0" y="489.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-21">
+<clipPath id="terminal-2104395871-line-21">
     <rect x="0" y="513.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-22">
+<clipPath id="terminal-2104395871-line-22">
     <rect x="0" y="538.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-23">
+<clipPath id="terminal-2104395871-line-23">
     <rect x="0" y="562.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-24">
+<clipPath id="terminal-2104395871-line-24">
     <rect x="0" y="587.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-25">
+<clipPath id="terminal-2104395871-line-25">
     <rect x="0" y="611.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-3451094975-line-26">
+<clipPath id="terminal-2104395871-line-26">
     <rect x="0" y="635.9" 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="731.2" rx="8"/><text class="terminal-3451094975-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;prepare-provider-packages</text>
+    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="731.2" rx="8"/><text class="terminal-2104395871-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;prepare-provider-packages</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-3451094975-clip-terminal)">
+    <g transform="translate(9, 41)" clip-path="url(#terminal-2104395871-clip-terminal)">
     
-    <g class="terminal-3451094975-matrix">
-    <text class="terminal-3451094975-r2" x="1464" y="20" textLength="12.2" clip-path="url(#terminal-3451094975-line-0)">
-</text><text class="terminal-3451094975-r3" x="12.2" y="44.4" textLength="85.4" clip-path="url(#terminal-3451094975-line-1)">Usage:&#160;</text><text class="terminal-3451094975-r1" x="97.6" y="44.4" textLength="1281" clip-path="url(#terminal-3451094975-line-1)">breeze&#160;prepare-provider-packages&#160;[OPTIONS]&#160;[airbyte&#160;|&#160;alibaba&#160;|&#160;amazon&#160;|&#160;apache.beam&#160;|&#160;apache.cassandra&#160;|</text><text class="terminal-3451094975-r2" x="1464" y="44.4" tex [...]
-</text><text class="terminal-3451094975-r1" x="12.2" y="68.8" textLength="1366.4" clip-path="url(#terminal-3451094975-line-2)">&#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;&#160;apache.drill&#160;|&#160;apache.druid&#160;|&#160;apache.hdfs&#160;|&#160;apache.hive&#160;|&#160;apache.kylin&#160;|</text>< [...]
-</text><text class="terminal-3451094975-r1" x="12.2" y="93.2" textLength="1354.2" clip-path="url(#terminal-3451094975-line-3)">&#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;&#160;apache.livy&#160;|&#160;apache.pig&#160;|&#160;apache.pinot&#160;|&#160;apache.spark&#160;|&#160;apache.sqoop&#160;|</text><t [...]
-</text><text class="terminal-3451094975-r1" x="12.2" y="117.6" textLength="1439.6" clip-path="url(#terminal-3451094975-line-4)">&#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;&#160;arangodb&#160;|&#160;asana&#160;|&#160;celery&#160;|&#160;cloudant&#160;|&#160;cncf.kubernetes&#160;|&#160;core.sql&#160;|&# [...]
-</text><text class="terminal-3451094975-r1" x="12.2" y="142" textLength="1439.6" clip-path="url(#terminal-3451094975-line-5)">&#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;&#160;|&#160;datadog&#160;|&#160;dbt.cloud&#160;|&#160;dingding&#160;|&#160;discord&#160;|&#160;docker&#160;|&#160;elasticsearch&#16 [...]
-</text><text class="terminal-3451094975-r1" x="12.2" y="166.4" textLength="1439.6" clip-path="url(#terminal-3451094975-line-6)">&#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;&#160;facebook&#160;|&#160;ftp&#160;|&#160;github&#160;|&#160;google&#160;|&#160;grpc&#160;|&#160;hashicorp&#160;|&#160;http&#160; [...]
-</text><text class="terminal-3451094975-r1" x="12.2" y="190.8" textLength="1415.2" clip-path="url(#terminal-3451094975-line-7)">&#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;&#160;jdbc&#160;|&#160;jenkins&#160;|&#160;jira&#160;|&#160;microsoft.azure&#160;|&#160;microsoft.mssql&#160;|&#160;microsoft.psrp [...]
-</text><text class="terminal-3451094975-r1" x="12.2" y="215.2" textLength="1427.4" clip-path="url(#terminal-3451094975-line-8)">&#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;&#160;microsoft.winrm&#160;|&#160;mongo&#160;|&#160;mysql&#160;|&#160;neo4j&#160;|&#160;odbc&#160;|&#160;openfaas&#160;|&#160;opsg [...]
-</text><text class="terminal-3451094975-r1" x="12.2" y="239.6" textLength="1354.2" clip-path="url(#terminal-3451094975-line-9)">&#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;&#160;|&#160;pagerduty&#160;|&#160;papermill&#160;|&#160;plexus&#160;|&#160;postgres&#160;|&#160;presto&#160;|&#160;qubole&#160;|& [...]
-</text><text class="terminal-3451094975-r1" x="12.2" y="264" textLength="1342" clip-path="url(#terminal-3451094975-line-10)">&#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;&#160;salesforce&#160;|&#160;samba&#160;|&#160;segment&#160;|&#160;sendgrid&#160;|&#160;sftp&#160;|&#160;singularity&#160;|&#160;slac [...]
-</text><text class="terminal-3451094975-r1" x="12.2" y="288.4" textLength="1403" clip-path="url(#terminal-3451094975-line-11)">&#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;&#160;snowflake&#160;|&#160;sqlite&#160;|&#160;ssh&#160;|&#160;tableau&#160;|&#160;tabular&#160;|&#160;telegram&#160;|&#160;trino&# [...]
-</text><text class="terminal-3451094975-r1" x="12.2" y="312.8" textLength="732" clip-path="url(#terminal-3451094975-line-12)">&#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;&#160;yandex&#160;|&#160;zendesk]...</text><text class="terminal-3451094975-r2" x="1464" y="312.8" textLength="12.2" clip-path="url( [...]
-</text><text class="terminal-3451094975-r2" x="1464" y="337.2" textLength="12.2" clip-path="url(#terminal-3451094975-line-13)">
-</text><text class="terminal-3451094975-r2" x="12.2" y="361.6" textLength="585.6" clip-path="url(#terminal-3451094975-line-14)">Prepare&#160;sdist/whl&#160;packages&#160;of&#160;Airflow&#160;Providers.</text><text class="terminal-3451094975-r2" x="1464" y="361.6" textLength="12.2" clip-path="url(#terminal-3451094975-line-14)">
-</text><text class="terminal-3451094975-r2" x="1464" y="386" textLength="12.2" clip-path="url(#terminal-3451094975-line-15)">
-</text><text class="terminal-3451094975-r4" x="0" y="410.4" textLength="24.4" clip-path="url(#terminal-3451094975-line-16)">╭─</text><text class="terminal-3451094975-r4" x="24.4" y="410.4" textLength="1415.2" clip-path="url(#terminal-3451094975-line-16)">&#160;Package&#160;flags&#160;─────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-3451094975-r4" x="1439.6" y="410.4" textLength="24.4" clip-path="url(#termina [...]
-</text><text class="terminal-3451094975-r4" x="0" y="434.8" textLength="12.2" clip-path="url(#terminal-3451094975-line-17)">│</text><text class="terminal-3451094975-r5" x="24.4" y="434.8" textLength="12.2" clip-path="url(#terminal-3451094975-line-17)">-</text><text class="terminal-3451094975-r5" x="36.6" y="434.8" textLength="97.6" clip-path="url(#terminal-3451094975-line-17)">-package</text><text class="terminal-3451094975-r5" x="134.2" y="434.8" textLength="85.4" clip-path="url(#termin [...]
-</text><text class="terminal-3451094975-r4" x="0" y="459.2" textLength="12.2" clip-path="url(#terminal-3451094975-line-18)">│</text><text class="terminal-3451094975-r5" x="24.4" y="459.2" textLength="12.2" clip-path="url(#terminal-3451094975-line-18)">-</text><text class="terminal-3451094975-r5" x="36.6" y="459.2" textLength="97.6" clip-path="url(#terminal-3451094975-line-18)">-version</text><text class="terminal-3451094975-r5" x="134.2" y="459.2" textLength="195.2" clip-path="url(#termi [...]
-</text><text class="terminal-3451094975-r4" x="0" y="483.6" textLength="12.2" clip-path="url(#terminal-3451094975-line-19)">│</text><text class="terminal-3451094975-r5" x="24.4" y="483.6" textLength="12.2" clip-path="url(#terminal-3451094975-line-19)">-</text><text class="terminal-3451094975-r5" x="36.6" y="483.6" textLength="97.6" clip-path="url(#terminal-3451094975-line-19)">-package</text><text class="terminal-3451094975-r5" x="134.2" y="483.6" textLength="122" clip-path="url(#termina [...]
-</text><text class="terminal-3451094975-r4" x="0" y="508" textLength="12.2" clip-path="url(#terminal-3451094975-line-20)">│</text><text class="terminal-3451094975-r5" x="24.4" y="508" textLength="12.2" clip-path="url(#terminal-3451094975-line-20)">-</text><text class="terminal-3451094975-r5" x="36.6" y="508" textLength="73.2" clip-path="url(#terminal-3451094975-line-20)">-debug</text><text class="terminal-3451094975-r2" x="378.2" y="508" textLength="878.4" clip-path="url(#terminal-345109 [...]
-</text><text class="terminal-3451094975-r4" x="0" y="532.4" textLength="1464" clip-path="url(#terminal-3451094975-line-21)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-3451094975-r2" x="1464" y="532.4" textLength="12.2" clip-path="url(#terminal-3451094975-line-21)">
-</text><text class="terminal-3451094975-r4" x="0" y="556.8" textLength="24.4" clip-path="url(#terminal-3451094975-line-22)">╭─</text><text class="terminal-3451094975-r4" x="24.4" y="556.8" textLength="1415.2" clip-path="url(#terminal-3451094975-line-22)">&#160;Options&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-3451094975-r4" x="1439.6" y="556.8" textLength="24.4" clip-path="url(#terminal-345 [...]
-</text><text class="terminal-3451094975-r4" x="0" y="581.2" textLength="12.2" clip-path="url(#terminal-3451094975-line-23)">│</text><text class="terminal-3451094975-r5" x="24.4" y="581.2" textLength="12.2" clip-path="url(#terminal-3451094975-line-23)">-</text><text class="terminal-3451094975-r5" x="36.6" y="581.2" textLength="97.6" clip-path="url(#terminal-3451094975-line-23)">-verbose</text><text class="terminal-3451094975-r7" x="280.6" y="581.2" textLength="24.4" clip-path="url(#termin [...]
-</text><text class="terminal-3451094975-r4" x="0" y="605.6" textLength="12.2" clip-path="url(#terminal-3451094975-line-24)">│</text><text class="terminal-3451094975-r5" x="24.4" y="605.6" textLength="12.2" clip-path="url(#terminal-3451094975-line-24)">-</text><text class="terminal-3451094975-r5" x="36.6" y="605.6" textLength="48.8" clip-path="url(#terminal-3451094975-line-24)">-dry</text><text class="terminal-3451094975-r5" x="85.4" y="605.6" textLength="48.8" clip-path="url(#terminal-34 [...]
-</text><text class="terminal-3451094975-r4" x="0" y="630" textLength="12.2" clip-path="url(#terminal-3451094975-line-25)">│</text><text class="terminal-3451094975-r5" x="24.4" y="630" textLength="12.2" clip-path="url(#terminal-3451094975-line-25)">-</text><text class="terminal-3451094975-r5" x="36.6" y="630" textLength="85.4" clip-path="url(#terminal-3451094975-line-25)">-github</text><text class="terminal-3451094975-r5" x="122" y="630" textLength="134.2" clip-path="url(#terminal-3451094 [...]
-</text><text class="terminal-3451094975-r4" x="0" y="654.4" textLength="12.2" clip-path="url(#terminal-3451094975-line-26)">│</text><text class="terminal-3451094975-r5" x="24.4" y="654.4" textLength="12.2" clip-path="url(#terminal-3451094975-line-26)">-</text><text class="terminal-3451094975-r5" x="36.6" y="654.4" textLength="61" clip-path="url(#terminal-3451094975-line-26)">-help</text><text class="terminal-3451094975-r7" x="280.6" y="654.4" textLength="24.4" clip-path="url(#terminal-34 [...]
-</text><text class="terminal-3451094975-r4" x="0" y="678.8" textLength="1464" clip-path="url(#terminal-3451094975-line-27)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-3451094975-r2" x="1464" y="678.8" textLength="12.2" clip-path="url(#terminal-3451094975-line-27)">
+    <g class="terminal-2104395871-matrix">
+    <text class="terminal-2104395871-r2" x="1464" y="20" textLength="12.2" clip-path="url(#terminal-2104395871-line-0)">
+</text><text class="terminal-2104395871-r3" x="12.2" y="44.4" textLength="85.4" clip-path="url(#terminal-2104395871-line-1)">Usage:&#160;</text><text class="terminal-2104395871-r1" x="97.6" y="44.4" textLength="1281" clip-path="url(#terminal-2104395871-line-1)">breeze&#160;prepare-provider-packages&#160;[OPTIONS]&#160;[airbyte&#160;|&#160;alibaba&#160;|&#160;amazon&#160;|&#160;apache.beam&#160;|&#160;apache.cassandra&#160;|</text><text class="terminal-2104395871-r2" x="1464" y="44.4" tex [...]
+</text><text class="terminal-2104395871-r1" x="12.2" y="68.8" textLength="1366.4" clip-path="url(#terminal-2104395871-line-2)">&#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;&#160;apache.drill&#160;|&#160;apache.druid&#160;|&#160;apache.hdfs&#160;|&#160;apache.hive&#160;|&#160;apache.kylin&#160;|</text>< [...]
+</text><text class="terminal-2104395871-r1" x="12.2" y="93.2" textLength="1354.2" clip-path="url(#terminal-2104395871-line-3)">&#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;&#160;apache.livy&#160;|&#160;apache.pig&#160;|&#160;apache.pinot&#160;|&#160;apache.spark&#160;|&#160;apache.sqoop&#160;|</text><t [...]
+</text><text class="terminal-2104395871-r1" x="12.2" y="117.6" textLength="1329.8" clip-path="url(#terminal-2104395871-line-4)">&#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;&#160;arangodb&#160;|&#160;asana&#160;|&#160;celery&#160;|&#160;cloudant&#160;|&#160;cncf.kubernetes&#160;|&#160;common.sql&#160;| [...]
+</text><text class="terminal-2104395871-r1" x="12.2" y="142" textLength="1439.6" clip-path="url(#terminal-2104395871-line-5)">&#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;&#160;databricks&#160;|&#160;datadog&#160;|&#160;dbt.cloud&#160;|&#160;dingding&#160;|&#160;discord&#160;|&#160;docker&#160;|&#160;e [...]
+</text><text class="terminal-2104395871-r1" x="12.2" y="166.4" textLength="1439.6" clip-path="url(#terminal-2104395871-line-6)">&#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;&#160;|&#160;exasol&#160;|&#160;facebook&#160;|&#160;ftp&#160;|&#160;github&#160;|&#160;google&#160;|&#160;grpc&#160;|&#160;hashic [...]
+</text><text class="terminal-2104395871-r1" x="12.2" y="190.8" textLength="1342" clip-path="url(#terminal-2104395871-line-7)">&#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;&#160;influxdb&#160;|&#160;jdbc&#160;|&#160;jenkins&#160;|&#160;jira&#160;|&#160;microsoft.azure&#160;|&#160;microsoft.mssql&#160;|< [...]
+</text><text class="terminal-2104395871-r1" x="12.2" y="215.2" textLength="1415.2" clip-path="url(#terminal-2104395871-line-8)">&#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;&#160;microsoft.psrp&#160;|&#160;microsoft.winrm&#160;|&#160;mongo&#160;|&#160;mysql&#160;|&#160;neo4j&#160;|&#160;odbc&#160;|&#16 [...]
+</text><text class="terminal-2104395871-r1" x="12.2" y="239.6" textLength="1366.4" clip-path="url(#terminal-2104395871-line-9)">&#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;&#160;opsgenie&#160;|&#160;oracle&#160;|&#160;pagerduty&#160;|&#160;papermill&#160;|&#160;plexus&#160;|&#160;postgres&#160;|&#160; [...]
+</text><text class="terminal-2104395871-r1" x="12.2" y="264" textLength="1427.4" clip-path="url(#terminal-2104395871-line-10)">&#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;&#160;qubole&#160;|&#160;redis&#160;|&#160;salesforce&#160;|&#160;samba&#160;|&#160;segment&#160;|&#160;sendgrid&#160;|&#160;sftp&# [...]
+</text><text class="terminal-2104395871-r1" x="12.2" y="288.4" textLength="1403" clip-path="url(#terminal-2104395871-line-11)">&#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;&#160;|&#160;slack&#160;|&#160;snowflake&#160;|&#160;sqlite&#160;|&#160;ssh&#160;|&#160;tableau&#160;|&#160;tabular&#160;|&#160;tel [...]
+</text><text class="terminal-2104395871-r1" x="12.2" y="312.8" textLength="854" clip-path="url(#terminal-2104395871-line-12)">&#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;&#160;vertica&#160;|&#160;yandex&#160;|&#160;zendesk]...</text><text class="terminal-2104395871-r2" x="1464" y="312.8" textLength="1 [...]
+</text><text class="terminal-2104395871-r2" x="1464" y="337.2" textLength="12.2" clip-path="url(#terminal-2104395871-line-13)">
+</text><text class="terminal-2104395871-r2" x="12.2" y="361.6" textLength="585.6" clip-path="url(#terminal-2104395871-line-14)">Prepare&#160;sdist/whl&#160;packages&#160;of&#160;Airflow&#160;Providers.</text><text class="terminal-2104395871-r2" x="1464" y="361.6" textLength="12.2" clip-path="url(#terminal-2104395871-line-14)">
+</text><text class="terminal-2104395871-r2" x="1464" y="386" textLength="12.2" clip-path="url(#terminal-2104395871-line-15)">
+</text><text class="terminal-2104395871-r4" x="0" y="410.4" textLength="24.4" clip-path="url(#terminal-2104395871-line-16)">╭─</text><text class="terminal-2104395871-r4" x="24.4" y="410.4" textLength="1415.2" clip-path="url(#terminal-2104395871-line-16)">&#160;Package&#160;flags&#160;─────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-2104395871-r4" x="1439.6" y="410.4" textLength="24.4" clip-path="url(#termina [...]
+</text><text class="terminal-2104395871-r4" x="0" y="434.8" textLength="12.2" clip-path="url(#terminal-2104395871-line-17)">│</text><text class="terminal-2104395871-r5" x="24.4" y="434.8" textLength="12.2" clip-path="url(#terminal-2104395871-line-17)">-</text><text class="terminal-2104395871-r5" x="36.6" y="434.8" textLength="97.6" clip-path="url(#terminal-2104395871-line-17)">-package</text><text class="terminal-2104395871-r5" x="134.2" y="434.8" textLength="85.4" clip-path="url(#termin [...]
+</text><text class="terminal-2104395871-r4" x="0" y="459.2" textLength="12.2" clip-path="url(#terminal-2104395871-line-18)">│</text><text class="terminal-2104395871-r5" x="24.4" y="459.2" textLength="12.2" clip-path="url(#terminal-2104395871-line-18)">-</text><text class="terminal-2104395871-r5" x="36.6" y="459.2" textLength="97.6" clip-path="url(#terminal-2104395871-line-18)">-version</text><text class="terminal-2104395871-r5" x="134.2" y="459.2" textLength="195.2" clip-path="url(#termi [...]
+</text><text class="terminal-2104395871-r4" x="0" y="483.6" textLength="12.2" clip-path="url(#terminal-2104395871-line-19)">│</text><text class="terminal-2104395871-r5" x="24.4" y="483.6" textLength="12.2" clip-path="url(#terminal-2104395871-line-19)">-</text><text class="terminal-2104395871-r5" x="36.6" y="483.6" textLength="97.6" clip-path="url(#terminal-2104395871-line-19)">-package</text><text class="terminal-2104395871-r5" x="134.2" y="483.6" textLength="122" clip-path="url(#termina [...]
+</text><text class="terminal-2104395871-r4" x="0" y="508" textLength="12.2" clip-path="url(#terminal-2104395871-line-20)">│</text><text class="terminal-2104395871-r5" x="24.4" y="508" textLength="12.2" clip-path="url(#terminal-2104395871-line-20)">-</text><text class="terminal-2104395871-r5" x="36.6" y="508" textLength="73.2" clip-path="url(#terminal-2104395871-line-20)">-debug</text><text class="terminal-2104395871-r2" x="378.2" y="508" textLength="878.4" clip-path="url(#terminal-210439 [...]
+</text><text class="terminal-2104395871-r4" x="0" y="532.4" textLength="1464" clip-path="url(#terminal-2104395871-line-21)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-2104395871-r2" x="1464" y="532.4" textLength="12.2" clip-path="url(#terminal-2104395871-line-21)">
+</text><text class="terminal-2104395871-r4" x="0" y="556.8" textLength="24.4" clip-path="url(#terminal-2104395871-line-22)">╭─</text><text class="terminal-2104395871-r4" x="24.4" y="556.8" textLength="1415.2" clip-path="url(#terminal-2104395871-line-22)">&#160;Options&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-2104395871-r4" x="1439.6" y="556.8" textLength="24.4" clip-path="url(#terminal-210 [...]
+</text><text class="terminal-2104395871-r4" x="0" y="581.2" textLength="12.2" clip-path="url(#terminal-2104395871-line-23)">│</text><text class="terminal-2104395871-r5" x="24.4" y="581.2" textLength="12.2" clip-path="url(#terminal-2104395871-line-23)">-</text><text class="terminal-2104395871-r5" x="36.6" y="581.2" textLength="97.6" clip-path="url(#terminal-2104395871-line-23)">-verbose</text><text class="terminal-2104395871-r7" x="280.6" y="581.2" textLength="24.4" clip-path="url(#termin [...]
+</text><text class="terminal-2104395871-r4" x="0" y="605.6" textLength="12.2" clip-path="url(#terminal-2104395871-line-24)">│</text><text class="terminal-2104395871-r5" x="24.4" y="605.6" textLength="12.2" clip-path="url(#terminal-2104395871-line-24)">-</text><text class="terminal-2104395871-r5" x="36.6" y="605.6" textLength="48.8" clip-path="url(#terminal-2104395871-line-24)">-dry</text><text class="terminal-2104395871-r5" x="85.4" y="605.6" textLength="48.8" clip-path="url(#terminal-21 [...]
+</text><text class="terminal-2104395871-r4" x="0" y="630" textLength="12.2" clip-path="url(#terminal-2104395871-line-25)">│</text><text class="terminal-2104395871-r5" x="24.4" y="630" textLength="12.2" clip-path="url(#terminal-2104395871-line-25)">-</text><text class="terminal-2104395871-r5" x="36.6" y="630" textLength="85.4" clip-path="url(#terminal-2104395871-line-25)">-github</text><text class="terminal-2104395871-r5" x="122" y="630" textLength="134.2" clip-path="url(#terminal-2104395 [...]
+</text><text class="terminal-2104395871-r4" x="0" y="654.4" textLength="12.2" clip-path="url(#terminal-2104395871-line-26)">│</text><text class="terminal-2104395871-r5" x="24.4" y="654.4" textLength="12.2" clip-path="url(#terminal-2104395871-line-26)">-</text><text class="terminal-2104395871-r5" x="36.6" y="654.4" textLength="61" clip-path="url(#terminal-2104395871-line-26)">-help</text><text class="terminal-2104395871-r7" x="280.6" y="654.4" textLength="24.4" clip-path="url(#terminal-21 [...]
+</text><text class="terminal-2104395871-r4" x="0" y="678.8" textLength="1464" clip-path="url(#terminal-2104395871-line-27)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-2104395871-r2" x="1464" y="678.8" textLength="12.2" clip-path="url(#terminal-2104395871-line-27)">
 </text>
     </g>
     </g>
diff --git a/newsfragments/NEW.significant.rst b/newsfragments/NEW.significant.rst
new file mode 100644
index 0000000000..01a1b678ed
--- /dev/null
+++ b/newsfragments/NEW.significant.rst
@@ -0,0 +1 @@
+The DB related classes: ``DBApiHook``, ``SQLSensor`` have been moved to ``apache-airflow-providers-common-sql`` provider.
diff --git a/scripts/ci/installed_providers.txt b/scripts/ci/installed_providers.txt
index c6b02bfae1..013fb587e3 100644
--- a/scripts/ci/installed_providers.txt
+++ b/scripts/ci/installed_providers.txt
@@ -1,6 +1,7 @@
 amazon
 celery
 cncf.kubernetes
+common.sql
 docker
 elasticsearch
 ftp
diff --git a/setup.py b/setup.py
index 991b6b0617..2435ac046e 100644
--- a/setup.py
+++ b/setup.py
@@ -654,6 +654,7 @@ EXTRAS_DEPENDENCIES = sort_extras_dependencies()
 # Those providers do not have dependency on airflow2.0 because that would lead to circular dependencies.
 # This is not a problem for PIP but some tools (pipdeptree) show those as a warning.
 PREINSTALLED_PROVIDERS = [
+    'common.sql',
     'ftp',
     'http',
     'imap',
diff --git a/tests/deprecated_classes.py b/tests/deprecated_classes.py
index c06adedbc1..522e62da72 100644
--- a/tests/deprecated_classes.py
+++ b/tests/deprecated_classes.py
@@ -19,10 +19,6 @@ HOOKS = [
         "airflow.hooks.base.BaseHook",
         "airflow.hooks.base_hook.BaseHook",
     ),
-    (
-        "airflow.hooks.dbapi.DbApiHook",
-        "airflow.hooks.dbapi_hook.DbApiHook",
-    ),
     (
         "airflow.providers.apache.cassandra.hooks.cassandra.CassandraHook",
         "airflow.contrib.hooks.cassandra_hook.CassandraHook",
diff --git a/tests/operators/test_generic_transfer.py b/tests/operators/test_generic_transfer.py
index 2ce165b126..b56666db79 100644
--- a/tests/operators/test_generic_transfer.py
+++ b/tests/operators/test_generic_transfer.py
@@ -74,7 +74,7 @@ class TestMySql(unittest.TestCase):
             )
             op.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)
 
-    @mock.patch('airflow.hooks.dbapi.DbApiHook.insert_rows')
+    @mock.patch('airflow.providers.common.sql.hooks.sql.DbApiHook.insert_rows')
     def test_mysql_to_mysql_replace(self, mock_insert):
         sql = "SELECT * FROM connection LIMIT 10;"
         op = GenericTransfer(
@@ -126,7 +126,7 @@ class TestPostgres(unittest.TestCase):
         )
         op.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)
 
-    @mock.patch('airflow.hooks.dbapi.DbApiHook.insert_rows')
+    @mock.patch('airflow.providers.common.sql.hooks.sql.DbApiHook.insert_rows')
     def test_postgres_to_postgres_replace(self, mock_insert):
         sql = "SELECT id, conn_id, conn_type FROM connection LIMIT 10;"
         op = GenericTransfer(
diff --git a/tests/providers/core/sql/__init__.py b/tests/providers/common/__init__.py
similarity index 100%
rename from tests/providers/core/sql/__init__.py
rename to tests/providers/common/__init__.py
diff --git a/tests/providers/core/sql/operators/__init__.py b/tests/providers/common/sql/__init__.py
similarity index 100%
rename from tests/providers/core/sql/operators/__init__.py
rename to tests/providers/common/sql/__init__.py
diff --git a/airflow/providers/core/__init__.py b/tests/providers/common/sql/hooks/__init__.py
similarity index 100%
copy from airflow/providers/core/__init__.py
copy to tests/providers/common/sql/hooks/__init__.py
diff --git a/tests/hooks/test_dbapi.py b/tests/providers/common/sql/hooks/test_dbapi.py
similarity index 99%
rename from tests/hooks/test_dbapi.py
rename to tests/providers/common/sql/hooks/test_dbapi.py
index ad5e7ba6af..a44fa57e07 100644
--- a/tests/hooks/test_dbapi.py
+++ b/tests/providers/common/sql/hooks/test_dbapi.py
@@ -23,8 +23,8 @@ from unittest import mock
 
 import pytest
 
-from airflow.hooks.dbapi import DbApiHook
 from airflow.models import Connection
+from airflow.providers.common.sql.hooks.sql import DbApiHook
 
 
 class TestDbApiHook(unittest.TestCase):
diff --git a/airflow/providers/core/__init__.py b/tests/providers/common/sql/operators/__init__.py
similarity index 100%
copy from airflow/providers/core/__init__.py
copy to tests/providers/common/sql/operators/__init__.py
diff --git a/tests/providers/core/sql/operators/test_sql.py b/tests/providers/common/sql/operators/test_sql.py
similarity index 97%
rename from tests/providers/core/sql/operators/test_sql.py
rename to tests/providers/common/sql/operators/test_sql.py
index d39e2d9fae..63ef78ba1f 100644
--- a/tests/providers/core/sql/operators/test_sql.py
+++ b/tests/providers/common/sql/operators/test_sql.py
@@ -19,7 +19,7 @@
 import pytest
 
 from airflow.exceptions import AirflowException
-from airflow.providers.core.sql.operators.sql import SQLColumnCheckOperator, SQLTableCheckOperator
+from airflow.providers.common.sql.operators.sql import SQLColumnCheckOperator, SQLTableCheckOperator
 
 
 class MockHook:
diff --git a/airflow/providers/core/__init__.py b/tests/providers/common/sql/sensors/__init__.py
similarity index 100%
copy from airflow/providers/core/__init__.py
copy to tests/providers/common/sql/sensors/__init__.py
diff --git a/tests/sensors/test_sql_sensor.py b/tests/providers/common/sql/sensors/test_sql.py
similarity index 93%
rename from tests/sensors/test_sql_sensor.py
rename to tests/providers/common/sql/sensors/test_sql.py
index 23c31aa170..9746c07361 100644
--- a/tests/sensors/test_sql_sensor.py
+++ b/tests/providers/common/sql/sensors/test_sql.py
@@ -23,7 +23,8 @@ import pytest
 
 from airflow.exceptions import AirflowException
 from airflow.models.dag import DAG
-from airflow.sensors.sql import DbApiHook, SqlSensor
+from airflow.providers.common.sql.hooks.sql import DbApiHook
+from airflow.providers.common.sql.sensors.sql import SqlSensor
 from airflow.utils.timezone import datetime
 from tests.providers.apache.hive import TestHiveEnvironment
 
@@ -86,7 +87,7 @@ class TestSqlSensor(TestHiveEnvironment):
         )
         op2.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)
 
-    @mock.patch('airflow.sensors.sql.BaseHook')
+    @mock.patch('airflow.providers.common.sql.sensors.sql.BaseHook')
     def test_sql_sensor_postgres_poke(self, mock_hook):
         op = SqlSensor(
             task_id='sql_sensor_check',
@@ -118,7 +119,7 @@ class TestSqlSensor(TestHiveEnvironment):
         mock_get_records.return_value = [['1']]
         assert op.poke(None)
 
-    @mock.patch('airflow.sensors.sql.BaseHook')
+    @mock.patch('airflow.providers.common.sql.sensors.sql.BaseHook')
     def test_sql_sensor_postgres_poke_fail_on_empty(self, mock_hook):
         op = SqlSensor(
             task_id='sql_sensor_check', conn_id='postgres_default', sql="SELECT 1", fail_on_empty=True
@@ -131,7 +132,7 @@ class TestSqlSensor(TestHiveEnvironment):
         with pytest.raises(AirflowException):
             op.poke(None)
 
-    @mock.patch('airflow.sensors.sql.BaseHook')
+    @mock.patch('airflow.providers.common.sql.sensors.sql.BaseHook')
     def test_sql_sensor_postgres_poke_success(self, mock_hook):
         op = SqlSensor(
             task_id='sql_sensor_check', conn_id='postgres_default', sql="SELECT 1", success=lambda x: x in [1]
@@ -149,7 +150,7 @@ class TestSqlSensor(TestHiveEnvironment):
         mock_get_records.return_value = [['1']]
         assert not op.poke(None)
 
-    @mock.patch('airflow.sensors.sql.BaseHook')
+    @mock.patch('airflow.providers.common.sql.sensors.sql.BaseHook')
     def test_sql_sensor_postgres_poke_failure(self, mock_hook):
         op = SqlSensor(
             task_id='sql_sensor_check', conn_id='postgres_default', sql="SELECT 1", failure=lambda x: x in [1]
@@ -165,7 +166,7 @@ class TestSqlSensor(TestHiveEnvironment):
         with pytest.raises(AirflowException):
             op.poke(None)
 
-    @mock.patch('airflow.sensors.sql.BaseHook')
+    @mock.patch('airflow.providers.common.sql.sensors.sql.BaseHook')
     def test_sql_sensor_postgres_poke_failure_success(self, mock_hook):
         op = SqlSensor(
             task_id='sql_sensor_check',
@@ -188,7 +189,7 @@ class TestSqlSensor(TestHiveEnvironment):
         mock_get_records.return_value = [[2]]
         assert op.poke(None)
 
-    @mock.patch('airflow.sensors.sql.BaseHook')
+    @mock.patch('airflow.providers.common.sql.sensors.sql.BaseHook')
     def test_sql_sensor_postgres_poke_failure_success_same(self, mock_hook):
         op = SqlSensor(
             task_id='sql_sensor_check',
@@ -208,7 +209,7 @@ class TestSqlSensor(TestHiveEnvironment):
         with pytest.raises(AirflowException):
             op.poke(None)
 
-    @mock.patch('airflow.sensors.sql.BaseHook')
+    @mock.patch('airflow.providers.common.sql.sensors.sql.BaseHook')
     def test_sql_sensor_postgres_poke_invalid_failure(self, mock_hook):
         op = SqlSensor(
             task_id='sql_sensor_check',
@@ -225,7 +226,7 @@ class TestSqlSensor(TestHiveEnvironment):
             op.poke(None)
         assert "self.failure is present, but not callable -> [1]" == str(ctx.value)
 
-    @mock.patch('airflow.sensors.sql.BaseHook')
+    @mock.patch('airflow.providers.common.sql.sensors.sql.BaseHook')
     def test_sql_sensor_postgres_poke_invalid_success(self, mock_hook):
         op = SqlSensor(
             task_id='sql_sensor_check',
diff --git a/tests/providers/microsoft/mssql/hooks/test_mssql.py b/tests/providers/microsoft/mssql/hooks/test_mssql.py
index 63e032406d..202c7f8749 100644
--- a/tests/providers/microsoft/mssql/hooks/test_mssql.py
+++ b/tests/providers/microsoft/mssql/hooks/test_mssql.py
@@ -31,7 +31,7 @@ PYMSSQL_CONN = Connection(host='ip', schema='share', login='username', password=
 class TestMsSqlHook(unittest.TestCase):
     @unittest.skipIf(PY38, "Mssql package not available when Python >= 3.8.")
     @mock.patch('airflow.providers.microsoft.mssql.hooks.mssql.MsSqlHook.get_conn')
-    @mock.patch('airflow.hooks.dbapi.DbApiHook.get_connection')
+    @mock.patch('airflow.providers.common.sql.hooks.sql.DbApiHook.get_connection')
     def test_get_conn_should_return_connection(self, get_connection, mssql_get_conn):
         get_connection.return_value = PYMSSQL_CONN
         mssql_get_conn.return_value = mock.Mock()
@@ -44,7 +44,7 @@ class TestMsSqlHook(unittest.TestCase):
 
     @unittest.skipIf(PY38, "Mssql package not available when Python >= 3.8.")
     @mock.patch('airflow.providers.microsoft.mssql.hooks.mssql.MsSqlHook.get_conn')
-    @mock.patch('airflow.hooks.dbapi.DbApiHook.get_connection')
+    @mock.patch('airflow.providers.common.sql.hooks.sql.DbApiHook.get_connection')
     def test_set_autocommit_should_invoke_autocommit(self, get_connection, mssql_get_conn):
         get_connection.return_value = PYMSSQL_CONN
         mssql_get_conn.return_value = mock.Mock()
@@ -59,7 +59,7 @@ class TestMsSqlHook(unittest.TestCase):
 
     @unittest.skipIf(PY38, "Mssql package not available when Python >= 3.8.")
     @mock.patch('airflow.providers.microsoft.mssql.hooks.mssql.MsSqlHook.get_conn')
-    @mock.patch('airflow.hooks.dbapi.DbApiHook.get_connection')
+    @mock.patch('airflow.providers.common.sql.hooks.sql.DbApiHook.get_connection')
     def test_get_autocommit_should_return_autocommit_state(self, get_connection, mssql_get_conn):
         get_connection.return_value = PYMSSQL_CONN
         mssql_get_conn.return_value = mock.Mock()
diff --git a/tests/providers/presto/hooks/test_presto.py b/tests/providers/presto/hooks/test_presto.py
index e6fd7c5ed0..61f6dcb911 100644
--- a/tests/providers/presto/hooks/test_presto.py
+++ b/tests/providers/presto/hooks/test_presto.py
@@ -234,7 +234,7 @@ class TestPrestoHook(unittest.TestCase):
 
         self.db_hook = UnitTestPrestoHook()
 
-    @patch('airflow.hooks.dbapi.DbApiHook.insert_rows')
+    @patch('airflow.providers.common.sql.hooks.sql.DbApiHook.insert_rows')
     def test_insert_rows(self, mock_insert_rows):
         table = "table"
         rows = [("hello",), ("world",)]
diff --git a/tests/providers/trino/hooks/test_trino.py b/tests/providers/trino/hooks/test_trino.py
index a9be6545bf..2fdad8b71e 100644
--- a/tests/providers/trino/hooks/test_trino.py
+++ b/tests/providers/trino/hooks/test_trino.py
@@ -195,7 +195,7 @@ class TestTrinoHook(unittest.TestCase):
 
         self.db_hook = UnitTestTrinoHook()
 
-    @patch('airflow.hooks.dbapi.DbApiHook.insert_rows')
+    @patch('airflow.providers.common.sql.hooks.sql.DbApiHook.insert_rows')
     def test_insert_rows(self, mock_insert_rows):
         table = "table"
         rows = [("hello",), ("world",)]
diff --git a/tests/providers/vertica/hooks/test_vertica.py b/tests/providers/vertica/hooks/test_vertica.py
index 23ee7e5a0a..de6fe56bcb 100644
--- a/tests/providers/vertica/hooks/test_vertica.py
+++ b/tests/providers/vertica/hooks/test_vertica.py
@@ -68,7 +68,7 @@ class TestVerticaHook(unittest.TestCase):
 
         self.db_hook = UnitTestVerticaHook()
 
-    @patch('airflow.hooks.dbapi.DbApiHook.insert_rows')
+    @patch('airflow.providers.common.sql.hooks.sql.DbApiHook.insert_rows')
     def test_insert_rows(self, mock_insert_rows):
         table = "table"
         rows = [("hello",), ("world",)]
diff --git a/airflow/providers/core/__init__.py b/tests/system/providers/common/__init__.py
similarity index 100%
copy from airflow/providers/core/__init__.py
copy to tests/system/providers/common/__init__.py
diff --git a/airflow/providers/core/__init__.py b/tests/system/providers/common/sql/__init__.py
similarity index 100%
rename from airflow/providers/core/__init__.py
rename to tests/system/providers/common/sql/__init__.py
diff --git a/airflow/providers/core/sql/example_dags/example_sql_column_table_check.py b/tests/system/providers/common/sql/example_sql_column_table_check.py
similarity index 89%
rename from airflow/providers/core/sql/example_dags/example_sql_column_table_check.py
rename to tests/system/providers/common/sql/example_sql_column_table_check.py
index e83f021765..034060ae8b 100644
--- a/airflow/providers/core/sql/example_dags/example_sql_column_table_check.py
+++ b/tests/system/providers/common/sql/example_sql_column_table_check.py
@@ -16,7 +16,7 @@
 # specific language governing permissions and limitations
 # under the License.
 from airflow import DAG
-from airflow.providers.core.sql.operators.sql import SQLColumnCheckOperator, SQLTableCheckOperator
+from airflow.providers.common.sql.operators.sql import SQLColumnCheckOperator, SQLTableCheckOperator
 from airflow.utils.dates import datetime
 
 AIRFLOW_DB_METADATA_TABLE = "ab_role"
@@ -75,3 +75,9 @@ with DAG(
     # [END howto_operator_sql_table_check]
 
     column_check >> row_count_check
+
+
+from tests.system.utils import get_test_run  # noqa: E402
+
+# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest)
+test_run = get_test_run(dag)