From 29e6511be6cc6156a33622fe99777745029dbd5c Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 12 May 2022 21:54:10 +0200 Subject: [PATCH 01/13] Add Tabular provider Co-authored-by: Kyle Bendickson --- .github/CODEOWNERS | 1 + .../airflow_providers_bug_report.yml | 1 + CONTRIBUTING.rst | 4 +- INSTALL | 4 +- airflow/providers/tabular/CHANGELOG.rst | 24 ++++ airflow/providers/tabular/__init__.py | 17 +++ .../tabular/example_dags/__init__.py | 17 +++ .../tabular/example_dags/example_tabular.py | 46 ++++++ airflow/providers/tabular/hooks/__init__.py | 17 +++ airflow/providers/tabular/hooks/tabular.py | 104 ++++++++++++++ airflow/providers/tabular/provider.yaml | 43 ++++++ airflow/utils/db.py | 8 ++ .../commits.rst | 27 ++++ .../connections.rst | 41 ++++++ .../index.rst | 70 +++++++++ .../installing-providers-from-sources.rst | 18 +++ docs/apache-airflow/extra-packages-ref.rst | 2 + docs/integration-logos/tabular/tabular.jpeg | Bin 0 -> 5652 bytes docs/spelling_wordlist.txt | 1 + images/breeze/output-build-docs.svg | 133 +++++++++--------- images/breeze/output-commands-hash.txt | 2 +- .../output-prepare-provider-documentation.svg | 75 +++++----- .../output-prepare-provider-packages.svg | 76 +++++----- setup.py | 1 + tests/providers/tabular/__init__.py | 16 +++ tests/providers/tabular/hooks/__init__.py | 16 +++ tests/providers/tabular/hooks/test_tabular.py | 42 ++++++ 27 files changed, 660 insertions(+), 146 deletions(-) create mode 100644 airflow/providers/tabular/CHANGELOG.rst create mode 100644 airflow/providers/tabular/__init__.py create mode 100644 airflow/providers/tabular/example_dags/__init__.py create mode 100644 airflow/providers/tabular/example_dags/example_tabular.py create mode 100644 airflow/providers/tabular/hooks/__init__.py create mode 100644 airflow/providers/tabular/hooks/tabular.py create mode 100644 airflow/providers/tabular/provider.yaml create mode 100644 docs/apache-airflow-providers-tabular/commits.rst create mode 100644 docs/apache-airflow-providers-tabular/connections.rst create mode 100644 docs/apache-airflow-providers-tabular/index.rst create mode 100644 docs/apache-airflow-providers-tabular/installing-providers-from-sources.rst create mode 100644 docs/integration-logos/tabular/tabular.jpeg create mode 100644 tests/providers/tabular/__init__.py create mode 100644 tests/providers/tabular/hooks/__init__.py create mode 100644 tests/providers/tabular/hooks/test_tabular.py diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 594f939e46bb4..97e2a903ede91 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -59,6 +59,7 @@ /airflow/providers/google/ @turbaszek /airflow/providers/snowflake/ @turbaszek @potiuk @mik-laj /airflow/providers/cncf/kubernetes @jedcunningham +/airflow/providers/tabular/ @Fokko /docs/apache-airflow-providers-cncf-kubernetes @jedcunningham # Dev tools diff --git a/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml b/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml index 2e5b0267b0ae7..91372e1a92d2e 100644 --- a/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml @@ -92,6 +92,7 @@ body: - sqlite - ssh - tableau + - tabular - telegram - trino - vertica diff --git a/CONTRIBUTING.rst b/CONTRIBUTING.rst index f862d33b1963a..2ba10ac8ae823 100644 --- a/CONTRIBUTING.rst +++ b/CONTRIBUTING.rst @@ -655,8 +655,8 @@ grpc, hashicorp, hdfs, hive, http, imap, influxdb, jdbc, jenkins, jira, kerberos 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, telegram, trino, vertica, -virtualenv, webhdfs, winrm, yandex, zendesk +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 ba2f5500475ee..28eeaae37da45 100644 --- a/INSTALL +++ b/INSTALL @@ -104,8 +104,8 @@ grpc, hashicorp, hdfs, hive, http, imap, influxdb, jdbc, jenkins, jira, kerberos 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, telegram, trino, vertica, -virtualenv, webhdfs, winrm, yandex, zendesk +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/providers/tabular/CHANGELOG.rst b/airflow/providers/tabular/CHANGELOG.rst new file mode 100644 index 0000000000000..1f10cb7ef7691 --- /dev/null +++ b/airflow/providers/tabular/CHANGELOG.rst @@ -0,0 +1,24 @@ + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +Changelog +--------- + +0.0.1 +..... + +Initial version of the provider. diff --git a/airflow/providers/tabular/__init__.py b/airflow/providers/tabular/__init__.py new file mode 100644 index 0000000000000..217e5db960782 --- /dev/null +++ b/airflow/providers/tabular/__init__.py @@ -0,0 +1,17 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/airflow/providers/tabular/example_dags/__init__.py b/airflow/providers/tabular/example_dags/__init__.py new file mode 100644 index 0000000000000..217e5db960782 --- /dev/null +++ b/airflow/providers/tabular/example_dags/__init__.py @@ -0,0 +1,17 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/airflow/providers/tabular/example_dags/example_tabular.py b/airflow/providers/tabular/example_dags/example_tabular.py new file mode 100644 index 0000000000000..3bfbc607537d4 --- /dev/null +++ b/airflow/providers/tabular/example_dags/example_tabular.py @@ -0,0 +1,46 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from datetime import datetime, timedelta + +from airflow import DAG +from airflow.operators.bash import BashOperator +from airflow.providers.tabular.hooks.tabular import TabularHook + +default_args = { + "owner": "airflow", + "depends_on_past": False, + "email": ["airflow@airflow.com"], + "email_on_failure": False, + "email_on_retry": False, +} + +bash_command = f""" +echo "Our token: {TabularHook().get_token_macro()}" +echo "Also as an environment variable:" +env | grep TOKEN +""" + +with DAG( + "tabular", default_args=default_args, start_date=datetime(2021, 1, 1), schedule_interval=timedelta(1) +) as dag: + # This also works for the SparkSubmit operator + BashOperator( + task_id="with_tabular_environment_variable", + bash_command=bash_command, + env={"TOKEN": TabularHook().get_token_macro()}, + ) diff --git a/airflow/providers/tabular/hooks/__init__.py b/airflow/providers/tabular/hooks/__init__.py new file mode 100644 index 0000000000000..217e5db960782 --- /dev/null +++ b/airflow/providers/tabular/hooks/__init__.py @@ -0,0 +1,17 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/airflow/providers/tabular/hooks/tabular.py b/airflow/providers/tabular/hooks/tabular.py new file mode 100644 index 0000000000000..d212156abb5c1 --- /dev/null +++ b/airflow/providers/tabular/hooks/tabular.py @@ -0,0 +1,104 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from typing import Any, Dict, Tuple + +import requests +from requests import HTTPError + +from airflow.hooks.base import BaseHook + +DEFAULT_TABULAR_URL = "https://api.tabulardata.io/ws/v1" + +TOKENS_ENDPOINT = "oauth/tokens" + + +class TabularHook(BaseHook): + """ + This hook acts as a base hook for tabular services. It offers the ability to generate temporary, + short-lived session tokens to use within Airflow submitted jobs. + + :param tabular_conn_id: The :ref:`Tabular connection id` + which refers to the information to connect to the Tabular OAuth service. + """ + + conn_name_attr = 'tabular_conn_id' + default_conn_name = "tabular_default" + conn_type = "tabular" + hook_name = "Tabular" + + @staticmethod + def get_connection_form_widgets() -> Dict[str, Any]: + """Returns connection widgets to add to connection form""" + from flask_appbuilder.fieldwidgets import BS3TextFieldWidget + from flask_babel import lazy_gettext + from wtforms import StringField + + return { + "extra__tabular__baseUrl": StringField( + lazy_gettext("Tabular Base URL"), widget=BS3TextFieldWidget() + ), + } + + @staticmethod + def get_ui_field_behaviour() -> Dict[str, Any]: + """Returns custom field behaviour""" + return { + "hidden_fields": ["schema", "port", "host"], + "relabeling": { + "login": "Tabular Client ID", + "password": "Tabular Client Secret", + }, + "placeholders": { + "login": "client_id (token credentials auth)", + "password": "secret (token credentials auth)", + "extra__tabular__baseUrl": DEFAULT_TABULAR_URL, + }, + } + + def __init__(self, tabular_conn_id: str = default_conn_name) -> None: + super().__init__() + self.conn_id = tabular_conn_id + + def test_connection(self) -> Tuple[bool, str]: + """Test the Tabular connection.""" + try: + self.get_conn() + return True, "Successfully fetched token from Tabular" + except HTTPError as e: + return False, f"HTTP Error: {e}" + except Exception as e: + return False, str(e) + + def get_conn(self) -> str: + """Obtain a short-lived access token via a client_id and client_secret.""" + conn = self.get_connection(self.conn_id) + extras = conn.extra_dejson + client_id = conn.login + client_secret = conn.password + + base_url = extras.get("extra__tabular__baseUrl", DEFAULT_TABULAR_URL) + headers = {"Content-Type": "application/x-www-form-urlencoded"} + data = {"client_id": client_id, "client_secret": client_secret} + + response = requests.post(f"{base_url}/{TOKENS_ENDPOINT}", data=data, headers=headers) + response.raise_for_status() + + return response.json()["access_token"] + + def get_token_macro(self): + return f'{{{{ conn.{self.conn_id}.get_hook().get_conn() }}}}' diff --git a/airflow/providers/tabular/provider.yaml b/airflow/providers/tabular/provider.yaml new file mode 100644 index 0000000000000..ae37da70a15d3 --- /dev/null +++ b/airflow/providers/tabular/provider.yaml @@ -0,0 +1,43 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +--- +package-name: apache-airflow-providers-tabular +name: Tabular +description: | + `Tabular `__ + +versions: + - 0.0.1 + +additional-dependencies: + - apache-airflow>=2.2.0 + +integrations: + - integration-name: Tabular + external-doc-url: https://tabular.io/docs/ + logo: /integration-logos/tabular/tabular.jpeg + tags: [software] + +hooks: + - integration-name: Tabular + python-modules: + - airflow.providers.tabular.hooks.tabular + +connection-types: + - hook-class-name: airflow.providers.tabular.hooks.tabular.TabularHook + connection-type: tabular diff --git a/airflow/utils/db.py b/airflow/utils/db.py index e64d8ba672ec6..86959338c5702 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -593,6 +593,14 @@ def create_default_connections(session: Session = NEW_SESSION): ), session, ) + merge_conn( + Connection( + conn_id="tabular_default", + conn_type="tabular", + extra='{"extra__tabular__baseUrl": "https://api.tabulardata.io/ws/v1"}', + ), + session, + ) merge_conn( Connection( conn_id="trino_default", diff --git a/docs/apache-airflow-providers-tabular/commits.rst b/docs/apache-airflow-providers-tabular/commits.rst new file mode 100644 index 0000000000000..ea5473ee0b94b --- /dev/null +++ b/docs/apache-airflow-providers-tabular/commits.rst @@ -0,0 +1,27 @@ + + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + +Package apache-airflow-providers-tabular +------------------------------------------------------ + +`Tabular `__ + + +This is detailed commit list of changes for versions provider package: ``tabular``. +For high-level changelog, see :doc:`package information including changelog `. diff --git a/docs/apache-airflow-providers-tabular/connections.rst b/docs/apache-airflow-providers-tabular/connections.rst new file mode 100644 index 0000000000000..2bbbe45e4116b --- /dev/null +++ b/docs/apache-airflow-providers-tabular/connections.rst @@ -0,0 +1,41 @@ + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +.. _howto/connection:tabular: + +Tabular Connection +================== + +The Tabular connection type enables connection to Tabular to request a short lived token to access the Apache Iceberg tables. This token can be injected as an environment variable, to be used with Trino, Spark, Flink or your favorite query engine that supports Apache Iceberg. + +Default Connection IDs +---------------------- + +Tabular Hook uses the parameter ``tabular_conn_id`` for Connection IDs and the value of the parameter as ``tabular_default`` by default. You can create multiple connections in case you want to switch between environments. + +Configuring the Connection +-------------------------- +Client ID + The Client ID from Tabular + +Client Secret + The Client Secret from Tabular + +Extra (optional, connection parameters) + Specify the extra parameters (as json dictionary) that can be used the Tabular connection. The following parameters out of the standard python parameters are supported: + + * ``base_url`` - Optional, sets the URL to the Tabular environment. By default `https://api.tabulardata.io/ws/v1` diff --git a/docs/apache-airflow-providers-tabular/index.rst b/docs/apache-airflow-providers-tabular/index.rst new file mode 100644 index 0000000000000..21ebfe37da9b6 --- /dev/null +++ b/docs/apache-airflow-providers-tabular/index.rst @@ -0,0 +1,70 @@ + + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +``apache-airflow-providers-tabular`` +==================================== + +Content +------- + +.. toctree:: + :maxdepth: 1 + :caption: Guides + + Connection types + +.. toctree:: + :maxdepth: 1 + :caption: Resources + + Example DAGs + PyPI Repository + Installing from sources + Python API <_api/airflow/providers/tabular/index> + +.. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME! + + +.. toctree:: + :maxdepth: 1 + :caption: Commits + + Detailed list of commits + + +Package apache-airflow-providers-tabular +------------------------------------------------------ + +`Tabular `__ + + +Release: 0.0.1 + +Provider package +---------------- + +This is a provider package for ``tabular`` provider. All classes for this provider package +are in ``airflow.providers.tabular`` python package. + +Installation +------------ + +You can install this package on top of an existing Airflow 2.1+ installation via +``pip install apache-airflow-providers-tabular`` + +.. include:: ../../airflow/providers/tabular/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-tabular/installing-providers-from-sources.rst b/docs/apache-airflow-providers-tabular/installing-providers-from-sources.rst new file mode 100644 index 0000000000000..1c90205d15b3a --- /dev/null +++ b/docs/apache-airflow-providers-tabular/installing-providers-from-sources.rst @@ -0,0 +1,18 @@ + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +.. include:: ../installing-providers-from-sources.rst diff --git a/docs/apache-airflow/extra-packages-ref.rst b/docs/apache-airflow/extra-packages-ref.rst index 6fbba68fd8f0c..c6d86de2225fb 100644 --- a/docs/apache-airflow/extra-packages-ref.rst +++ b/docs/apache-airflow/extra-packages-ref.rst @@ -200,6 +200,8 @@ Those are extras that add dependencies needed for integration with external serv +---------------------+-----------------------------------------------------+-----------------------------------------------------+ | tableau | ``pip install 'apache-airflow[tableau]'`` | Tableau hooks and operators | +---------------------+-----------------------------------------------------+-----------------------------------------------------+ +| tabular | ``pip install 'apache-airflow[tabular]'`` | Tabular hooks | ++---------------------+-----------------------------------------------------+-----------------------------------------------------+ | telegram | ``pip install 'apache-airflow[telegram]'`` | Telegram hooks and operators | +---------------------+-----------------------------------------------------+-----------------------------------------------------+ | vertica | ``pip install 'apache-airflow[vertica]'`` | Vertica hook support as an Airflow backend | diff --git a/docs/integration-logos/tabular/tabular.jpeg b/docs/integration-logos/tabular/tabular.jpeg new file mode 100644 index 0000000000000000000000000000000000000000..e5ef939fcc982e75f9aada3dc0f989c948e49746 GIT binary patch literal 5652 zcmb7IcQjmIw7!^O^cgiG>V(mICt-vnjNSziM0BD?glIvc6TKush%Q=0kBBlx$*9qz z_fB-)__g)cdVjq=d#!uc+Gn5H=eytDXP+&}MRlTcn7r?^<06g3exEu%W00j8> zSBeu5PQef`7z6^75D^hV$VkY@NJ&UZ$tkFy!w<{A|Z zEzQ*;cmy~f5SSPYCZ-`LC8zm+%jI_f1_3&OP69l303U`&0K>cd0We*K3C6pM_g}%s zBLEW;fgr>C306rKO3L(+u7(hmVGeQZVxUHUi{MQ@z#G4;DJ~?crWEoy{ zIeC{^+V$>qG$qP z^i1H8R)i(wLhkBhSP*MyM{>xtlZWAP@9q1ghd=z9x6xx|bI@cfBNq5=he4jc@8T1& zhUqDsyH{odGqTc~;aJI>YWodkbKr>0M|pj$wu|qbDLR33qm**ERiTHTP0AyF$!V0j zcZuYh8&&HDj5`<1CvQ5V%+GR>8H zS!>)10KSAMi<+Dl2c;UHfzQ&Dnb!iMQRarD;+H_m3nNvXk6V#NM(vk?-a@A4xK>zM z9{vxxVb7kV%)3NOWwha{yg|XUe!KwD?yy zFPpN02p|2#C1AKbyFr=$v{Ap=;I*e}xD~J3%Ob2g1h#;o%X12>v$U zgTOe>5)whFScTYd335|dRE&-tt|g*?#8DA~qbMF2{}S+8*bGha#8iE7?_`ngo6Z<_ zRyg=NlDKL#xfy13JR#dZByjBDXH}|nfX!Gi&JO3wlE|pe^#0|9-Y@>Sh05@i&9HY& z_AM`so96VH*GVkb(^jU@Hp|a`wZx6F>u_p6i_vIBWoiho$x9x&No<&eU~%@p)X<){ zf~LeiECJKtqvq6mEW?Fc{z~g{{-xNp-cL<2JIAYB*^vlop>g|H9ktr8lb=%)9w>V6 zb-$U@P;-q6+waA$YYbnx+%ca&ahhFqZ8oPK3N~l5E>_YNBX%kPzgQCENAqWtq?#2~ zy3G3^_K%WlAT$kSRi$>hv{RweokxSkH!;B*gEsaR+GctB=Z*QLJ(qz0@2>J-*=-xQ zikD3Z+Dg6h7-b*H^GiT&qn+|gOJ=)4ae}t;iyo`B)#<}?+Dm}v;IGJgUfAL$DO;^6 z(->CMXe+&CDy>-!f>Dj)Kd?jo&R`e}A&GX*^|3kFz)i@dF~nQ~jrlmwvJ?GX=joC< z$-Vq2Su;;OW69l71pWe()9r)LE+RW$yqCDUL#I>sbMF68IZ%}@hYuD!%Sqm3E)_OU z-IeQuckZzzm9;fVQ=sVSR6MPt<2CaKL^vncrt0}BSAFLKx7gz2_$n}b6`f~(axw5u+wm704gP_18m^HD=8D+v zJdYla*9mh}v297le#N+nF^2t}7orGG<^54^%(X>qT@yDm6D7~>6#}N5rnoM9$^Biw z+M4k06;Uuppb%Ar*ml7P!OIklt?M3sU9UHj+e8?U_d5LRt_Tdn5f~4j089k_hu2qi z6b1qdL#aeq+1QcV)HI?Bimq|AVmcP(e-Rj0W##dXA+CtdBd2PLS98;%MeA7uJC9bJ z&QAwsLXjG_y0+W$O`=_S{k!bH%tz^x(r#riaU84vi4hy3NR1%usXQ;rwI3=n_!up% zEcYN8^=6u^KMA3Dac?KAm0DrnAR;a?*ezNO{w$_T`s>^p=>eI`Vk)oojc$pO2SYw% z-kB(#+UJ+RkM-2)!M$Upx6M>`^zS)78g)pnmOL0dNgw?T^Uq%7^+ajj_}XJoVsEBC z*R3C4pp&bbM7c5Ep*`7PhZ`sIsACd~$?QcsHzta+4`uUra?J|dS;Gp@V+~fhx#Sj2 zj#}N_q3)Gc>du+6_+F@Jbgz32FPHZa&jVA9Xm+!uwjOyE4UdvZ24-LE)^rGuE+vEZ zEK2acmR)yD8{+u}o!BNUuxPTQ@X&Sj6`3L@!6V;f9OhzIm;(f05E1eJ!JJhDNl48W zh$|aH3fivhqKY~eapl!GAn{&)||+Afs&TtdbxNO*>dUro5Mluj$O+CNAS3#k)g1qK2n7c1vB z?yQ^_8Pv<85y|BILwB5??z)yW9|l!BR8GV{cgpI>U~ztnIbwVviF+M(sn2?0{ zA132i4S-;*Y}#NdYIY$DSGPD}5mBUKAe07vQ%pg}G7GC^Y_2 zbU7)aZu#?a4prJ3fdtR-!U`3LWHNL08bU!!aL zqw80@xlZ+ba(I}BjHC7pTGWjC`z9keCeK1|G5Nx?W|iD^BntUWteRvVZ$~G>qATLl zY%3~Fz86K)e^*SokH!K-)6BtP2NC#JX6o|&TT8lSx8&$eo(lSxyTzn1;b2w@U>3(hH^kS)4s zW-yepLfbBvfyprtkLAd-rDV46aL|kr7a1U!)`#kHjlT_VPD^3S%~q!uOZQG`Px0$W z^chlnW{}XfWUYI;LK)d75+o}%_zoWupX6r!uz!yl#4*D)y8fB#sKq2gH{w?Ab#;x^ zR>b2W7|y%?!?e%STIG$~t}($&Dc>$*j^RG`V*QKaTvkKD-16+RoFHBedmYYU8I zw7+Z1TKt!Y+UF+}i4K8WIS+egytqUoaxE;MkNTUk47(TcEFkPIkiYgE_-yQZC+$pYTKod5kOL# z$rw=c%X!bWm%%#V|2U{DL1%V4)v%$E=)2sMc3xj~gbPbvGcF-`pa8C5fQj({T=)C8 z;KA5fg^}8UP+T^|WtF#VV->V!cK+>3E^-u7uPtDP-airH@Qw!H?%u9BF54Wan3ZC-rHF9g*6Dm+9{m9_*#`7J$3hN*U}Aky;0KCjR(e- z%G-;kdr0mxbxUl^u^a?Nf}r=2Alfngo~+n6*`E=O^In?xgwctglpZXa>!$jn()n;x z{#Hd0H1JVRcc^d&S@HXEzF~`z`0Rg7NDwO$0#zgdw5mKR54>Nq2GOr{Mo*AE{3cN1 zeJ#X|88h&fup#h-b;u={Tv9g8(t#xBH5)!euqP^I6|M8exBZ;)oV0M*asN4!JmLwA zm=?`zYwQdI(|MQI?NN5#N3ucaSKE7fH&Z{zB>fB|e$3{`in6%P_w0@Jhf6@|`TfVv zCimHUzOY7pB>iJm+lVDGb*y8RwsSTH!5$e(okduH9FJ2+9S3D@JQ@k?Zg|jnwI{Id z!Vj3$qK&moI&LE-_z)%yj)(ky-RlRuY&bI=GWKiRmsc4nbuM4bBaai?s=}9}z>`RD z;KTdN71dDX*dWN*kEl6)sPbpHyTqI&WKxo2OP{XTLn*f^Q>#ASZsNsqvr1uV@FieX zyLkF5H?3h-G^E>)BFF@ZA`L7)Ja(5F-8HOWR>yAD{b8v{jh?Hb(BLyqCCm?pWQXib zz`dLWsU9;fM0OfV?z?15^?mcyE4a_7HuMX&O8OeNg=atV3opJ{&|fD_FFfOvrU__0 zLkm6C5|qOm8vp*0!>)+CseSVKkHLJ;yB>Gyev8;x zY-I77)C${?KVy-(*W%`-^(-(`dnfkK4gYJDR4Hx56R(#MZ#8D(?S&sIxmK}yy;+ZP z@a7mUJ2b~J`>(4XaMhYa_xj}iQw%M~v%a&vKG~w0K0b1%wJo`Wg*o$MBB6tZ?K_lv zy%9n;nD*M_Mwkr@n)^=sgch3_0&5sF63Fikv9Y@J7M{$B6SFO`=2y-c;!Q7%YR|IU5USo}r3!`K}1UT_pv zazof*-+hN?$~rkpPL%}C_qkP^P&$6+UaRg_x397Mu+v>Kg5QzY;(qD^u{i9Zc<)<( zn3`_ofzZ!|YmX@lKP@>kD6~`N_Go#k6*hk0k8X##vLC_8WdE3r7`24>S^0bc5G74U ztt98k2h|bz&jMm^T^PTRdv;Kb@dc?7Bt5CWWkhX7401qC^5~VMDe+1Y2)-URY-Dlm zeL6qA97@**Z<)fg&kLWf`f+_v+&F1JE=jqtF-+i=SF_k{#LkbQES}jfk&{r2sVU2r z?az!d&8>&y7YG!51-X{b@6OYHH2mU%9tR^Q42L2{u2Y>y0H}(!X;&oWzXYTbgQ?K-)@&8uAxR`+;hcoZ(FFmMGHH#qwwEp0KEhxRn?7L=xkcV`ht(| z(C!H99BYNubkTcHpR?*y%xoX4v&2uSe)H}A#H+Gs^iJ~vObDJql&Akr@cS(9F}t6b z{3?6s&)}QF4JGG>u55ag8N8BT;LTW58s_WqC4ga0(IYpLdDi`0o_elP=1&dtkKBO( zqs$0jYWDSSBhOwYvLud`O!~9uL#`F9DD0}Y4){4N2ol+xIcn-c?m3Z8JbAv@=rml( zvsiRp!q6jtuI|NIS3ZE(4yuu?3f#PFP9)_ywmj0zlaOPG$iYfHPb85JAc&-(fuQoU zPIF^4`6CXSc?k9o?^-_*9=-%PI0SXisour-|4B?d$P+eSGF)9COV}RNCOhamY8hA) zr`vT49?E*0ekNvcSti64l}d2xR%4Fy%S1|Ttej-4lvo1YNa{oSe4EFxyj z$29a1whF5Ib%MchrnfOUN$r#0uMv+DOYJ$X|7IPf%Wia}^nJ@N0Wdk9%POjJ794RU z4s+&Ctiu~Cr>GFwx72& z6;+yzKD@uXKlZ4GOH1do9mc%JX1ST?Zckv!F*c=MQua9pUx*}g^WnB{qgPIK+K#y3 zIR(WM?L_LUPX+N8f;$0vcxf$>?X->Zf#gC){F7?nN-`uAeNN{x%I?H?|3KSFh4W~3 z`qyts&iRS-xNAVZQIrgrqR7Vz9c=7Zr1^~nO*Mt6;nT)~Ji>*YlK(vM@tT`eRu((* zU+)IDncuVtkH|Qd{gJhe67c#C!F59Jvn#(^{B81|=Bz;nEMw|FIAa+(NNM^bnR^!k ze7`LdD%iSfPg}bT2OmPYF;;Y6Zktiei5^Da5L>4*&Z)f#;5-;SK?i!fw=F8R67u(w83dW|nQ7jACt3@Kj6Y zMV#+Jf$X)ZY}&7s>mmX3d3sejhtl(S>)9yAHo+#+qqJ4`%#c$(omF6XobWn^>%H*q JB_Mt|`aih2BvAkW literal 0 HcmV?d00001 diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index eb8f6d7795c27..72e96bf3b0f26 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -392,6 +392,7 @@ TCP TLS TTY TZ +Tabular TaskDecorator TaskFlow TaskGroup diff --git a/images/breeze/output-build-docs.svg b/images/breeze/output-build-docs.svg index c65181b9aeff9..4e01061e4f330 100644 --- a/images/breeze/output-build-docs.svg +++ b/images/breeze/output-build-docs.svg @@ -1,4 +1,4 @@ - + - Command: build-docs + Command: build-docs -                                                                                                                          - Usage: breeze build-docs [OPTIONS]                                                                                      -                                                                                                                         - Build documentation in the container.                                                                                   -                                                                                                                         -╭─ Doc flags ──────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --docs-only        -d  Only build documentation.                                                                     -  --spellcheck-only  -s  Only run spell checking.                                                                      -  --for-production   -p  Builds documentation for official release i.e. all links point to stable version.             -  --package-filter   -p  List of packages to consider.                                                                 -                         (apache-airflow | apache-airflow-providers | apache-airflow-providers-airbyte |               -                         apache-airflow-providers-alibaba | apache-airflow-providers-amazon |                          -                         apache-airflow-providers-apache-beam | apache-airflow-providers-apache-cassandra |            -                         apache-airflow-providers-apache-drill | apache-airflow-providers-apache-druid |               -                         apache-airflow-providers-apache-hdfs | apache-airflow-providers-apache-hive |                 -                         apache-airflow-providers-apache-kylin | apache-airflow-providers-apache-livy |                -                         apache-airflow-providers-apache-pig | apache-airflow-providers-apache-pinot |                 -                         apache-airflow-providers-apache-spark | apache-airflow-providers-apache-sqoop |               -                         apache-airflow-providers-arangodb | apache-airflow-providers-asana |                          -                         apache-airflow-providers-celery | apache-airflow-providers-cloudant |                         -                         apache-airflow-providers-cncf-kubernetes | apache-airflow-providers-databricks |              -                         apache-airflow-providers-datadog | apache-airflow-providers-dbt-cloud |                       -                         apache-airflow-providers-dingding | apache-airflow-providers-discord |                        -                         apache-airflow-providers-docker | apache-airflow-providers-elasticsearch |                    -                         apache-airflow-providers-exasol | apache-airflow-providers-facebook |                         -                         apache-airflow-providers-ftp | apache-airflow-providers-github |                              -                         apache-airflow-providers-google | apache-airflow-providers-grpc |                             -                         apache-airflow-providers-hashicorp | apache-airflow-providers-http |                          -                         apache-airflow-providers-imap | apache-airflow-providers-influxdb |                           -                         apache-airflow-providers-jdbc | apache-airflow-providers-jenkins |                            -                         apache-airflow-providers-jira | apache-airflow-providers-microsoft-azure |                    -                         apache-airflow-providers-microsoft-mssql | apache-airflow-providers-microsoft-psrp |          -                         apache-airflow-providers-microsoft-winrm | apache-airflow-providers-mongo |                   -                         apache-airflow-providers-mysql | apache-airflow-providers-neo4j |                             -                         apache-airflow-providers-odbc | apache-airflow-providers-openfaas |                           -                         apache-airflow-providers-opsgenie | apache-airflow-providers-oracle |                         -                         apache-airflow-providers-pagerduty | apache-airflow-providers-papermill |                     -                         apache-airflow-providers-plexus | apache-airflow-providers-postgres |                         -                         apache-airflow-providers-presto | apache-airflow-providers-qubole |                           -                         apache-airflow-providers-redis | apache-airflow-providers-salesforce |                        -                         apache-airflow-providers-samba | apache-airflow-providers-segment |                           -                         apache-airflow-providers-sendgrid | apache-airflow-providers-sftp |                           -                         apache-airflow-providers-singularity | apache-airflow-providers-slack |                       -                         apache-airflow-providers-snowflake | apache-airflow-providers-sqlite |                        -                         apache-airflow-providers-ssh | apache-airflow-providers-tableau |                             -                         apache-airflow-providers-telegram | apache-airflow-providers-trino |                          -                         apache-airflow-providers-vertica | apache-airflow-providers-yandex |                          -                         apache-airflow-providers-zendesk | docker-stack | helm-chart)                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +                                                                                                                          + Usage: breeze build-docs [OPTIONS]                                                                                      +                                                                                                                         + Build documentation in the container.                                                                                   +                                                                                                                         +╭─ Doc flags ──────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +  --docs-only        -d  Only build documentation.                                                                     +  --spellcheck-only  -s  Only run spell checking.                                                                      +  --for-production   -p  Builds documentation for official release i.e. all links point to stable version.             +  --package-filter   -p  List of packages to consider.                                                                 +                         (apache-airflow | apache-airflow-providers | apache-airflow-providers-airbyte |               +                         apache-airflow-providers-alibaba | apache-airflow-providers-amazon |                          +                         apache-airflow-providers-apache-beam | apache-airflow-providers-apache-cassandra |            +                         apache-airflow-providers-apache-drill | apache-airflow-providers-apache-druid |               +                         apache-airflow-providers-apache-hdfs | apache-airflow-providers-apache-hive |                 +                         apache-airflow-providers-apache-kylin | apache-airflow-providers-apache-livy |                +                         apache-airflow-providers-apache-pig | apache-airflow-providers-apache-pinot |                 +                         apache-airflow-providers-apache-spark | apache-airflow-providers-apache-sqoop |               +                         apache-airflow-providers-arangodb | apache-airflow-providers-asana |                          +                         apache-airflow-providers-celery | apache-airflow-providers-cloudant |                         +                         apache-airflow-providers-cncf-kubernetes | apache-airflow-providers-databricks |              +                         apache-airflow-providers-datadog | apache-airflow-providers-dbt-cloud |                       +                         apache-airflow-providers-dingding | apache-airflow-providers-discord |                        +                         apache-airflow-providers-docker | apache-airflow-providers-elasticsearch |                    +                         apache-airflow-providers-exasol | apache-airflow-providers-facebook |                         +                         apache-airflow-providers-ftp | apache-airflow-providers-github |                              +                         apache-airflow-providers-google | apache-airflow-providers-grpc |                             +                         apache-airflow-providers-hashicorp | apache-airflow-providers-http |                          +                         apache-airflow-providers-imap | apache-airflow-providers-influxdb |                           +                         apache-airflow-providers-jdbc | apache-airflow-providers-jenkins |                            +                         apache-airflow-providers-jira | apache-airflow-providers-microsoft-azure |                    +                         apache-airflow-providers-microsoft-mssql | apache-airflow-providers-microsoft-psrp |          +                         apache-airflow-providers-microsoft-winrm | apache-airflow-providers-mongo |                   +                         apache-airflow-providers-mysql | apache-airflow-providers-neo4j |                             +                         apache-airflow-providers-odbc | apache-airflow-providers-openfaas |                           +                         apache-airflow-providers-opsgenie | apache-airflow-providers-oracle |                         +                         apache-airflow-providers-pagerduty | apache-airflow-providers-papermill |                     +                         apache-airflow-providers-plexus | apache-airflow-providers-postgres |                         +                         apache-airflow-providers-presto | apache-airflow-providers-qubole |                           +                         apache-airflow-providers-redis | apache-airflow-providers-salesforce |                        +                         apache-airflow-providers-samba | apache-airflow-providers-segment |                           +                         apache-airflow-providers-sendgrid | apache-airflow-providers-sftp |                           +                         apache-airflow-providers-singularity | apache-airflow-providers-slack |                       +                         apache-airflow-providers-snowflake | apache-airflow-providers-sqlite |                        +                         apache-airflow-providers-ssh | apache-airflow-providers-tableau |                             +                         apache-airflow-providers-tabular | apache-airflow-providers-telegram |                        +                         apache-airflow-providers-trino | apache-airflow-providers-vertica |                           +                         apache-airflow-providers-yandex | apache-airflow-providers-zendesk | docker-stack |           +                         helm-chart)                                                                                   +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +  --verbose            -v  Print verbose information about performed steps.                                            +  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 +  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           +  --help               -h  Show this message and exit.                                                                 +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt index 1030cc5119d5e..245412e972ccb 100644 --- a/images/breeze/output-commands-hash.txt +++ b/images/breeze/output-commands-hash.txt @@ -1 +1 @@ -513dc61480a9f3cc7619844251d7d6c4 +08aaddd549e3cfe7290a64609f4252fc diff --git a/images/breeze/output-prepare-provider-documentation.svg b/images/breeze/output-prepare-provider-documentation.svg index 2e5369e5cece8..47138dc074c4f 100644 --- a/images/breeze/output-prepare-provider-documentation.svg +++ b/images/breeze/output-prepare-provider-documentation.svg @@ -1,4 +1,4 @@ - + - Command: prepare-provider-documentation + Command: prepare-provider-documentation -                                                                                                                          - Usage: breeze prepare-provider-documentation [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra |   -                                              apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin |   -                                              apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop |    -                                              arangodb | asana | celery | cloudant | cncf.kubernetes | databricks |      -                                              datadog | dbt.cloud | dingding | discord | docker | elasticsearch |        -                                              exasol | facebook | ftp | github | google | grpc | hashicorp | http |      -                                              imap | influxdb | jdbc | jenkins | jira | microsoft.azure |                -                                              microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql |       -                                              neo4j | odbc | openfaas | opsgenie | oracle | pagerduty | papermill |      -                                              plexus | postgres | presto | qubole | redis | salesforce | samba |         -                                              segment | sendgrid | sftp | singularity | slack | snowflake | sqlite |     -                                              ssh | tableau | telegram | trino | vertica | yandex | zendesk]...          -                                                                                                                         - Prepare CHANGELOG, README and COMMITS information for providers.                                                        -                                                                                                                         -╭─ Provider documentation preparation flags ───────────────────────────────────────────────────────────────────────────╮ -  --debug    Drop user in shell instead of running the command. Useful for debugging.                                  -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --answer             -a  Force answer to questions. (y | n | q | yes | no | quit)                                    -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +                                                                                                                          + Usage: breeze prepare-provider-documentation [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra |   +                                              apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin |   +                                              apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop |    +                                              arangodb | asana | celery | cloudant | cncf.kubernetes | databricks |      +                                              datadog | dbt.cloud | dingding | discord | docker | elasticsearch |        +                                              exasol | facebook | ftp | github | google | grpc | hashicorp | http |      +                                              imap | influxdb | jdbc | jenkins | jira | microsoft.azure |                +                                              microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql |       +                                              neo4j | odbc | openfaas | opsgenie | oracle | pagerduty | papermill |      +                                              plexus | postgres | presto | qubole | redis | salesforce | samba |         +                                              segment | sendgrid | sftp | singularity | slack | snowflake | sqlite |     +                                              ssh | tableau | tabular | telegram | trino | vertica | yandex |            +                                              zendesk]...                                                                +                                                                                                                         + Prepare CHANGELOG, README and COMMITS information for providers.                                                        +                                                                                                                         +╭─ Provider documentation preparation flags ───────────────────────────────────────────────────────────────────────────╮ +  --debug    Drop user in shell instead of running the command. Useful for debugging.                                  +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +  --verbose            -v  Print verbose information about performed steps.                                            +  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 +  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           +  --answer             -a  Force answer to questions. (y | n | q | yes | no | quit)                                    +  --help               -h  Show this message and exit.                                                                 +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output-prepare-provider-packages.svg b/images/breeze/output-prepare-provider-packages.svg index ec2a04598f2c7..5d0f70952a1c5 100644 --- a/images/breeze/output-prepare-provider-packages.svg +++ b/images/breeze/output-prepare-provider-packages.svg @@ -19,63 +19,63 @@ font-weight: 700; } - .terminal-1292600021-matrix { + .terminal-857376060-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 26.400000000000002px; font-variant-east-asian: full-width; } - .terminal-1292600021-title { + .terminal-857376060-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-1292600021-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-1292600021-r2 { fill: #c5c8c6 } -.terminal-1292600021-r3 { fill: #d0b344;font-weight: bold } -.terminal-1292600021-r4 { fill: #868887 } -.terminal-1292600021-r5 { fill: #68a0b3;font-weight: bold } -.terminal-1292600021-r6 { fill: #8d7b39 } -.terminal-1292600021-r7 { fill: #98a84b;font-weight: bold } + .terminal-857376060-r1 { fill: #c5c8c6;font-weight: bold } +.terminal-857376060-r2 { fill: #c5c8c6 } +.terminal-857376060-r3 { fill: #d0b344;font-weight: bold } +.terminal-857376060-r4 { fill: #868887 } +.terminal-857376060-r5 { fill: #68a0b3;font-weight: bold } +.terminal-857376060-r6 { fill: #8d7b39 } +.terminal-857376060-r7 { fill: #98a84b;font-weight: bold } - Command: prepare-provider-packages + Command: prepare-provider-packages -                                                                                                                          - Usage: breeze prepare-provider-packages [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra |        -                                         apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin |        -                                         apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop |         -                                         arangodb | asana | celery | cloudant | cncf.kubernetes | databricks | datadog   -                                         | dbt.cloud | dingding | discord | docker | elasticsearch | exasol | facebook   -                                         | ftp | github | google | grpc | hashicorp | http | imap | influxdb | jdbc |    -                                         jenkins | jira | microsoft.azure | microsoft.mssql | microsoft.psrp |           -                                         microsoft.winrm | mongo | mysql | neo4j | odbc | openfaas | opsgenie | oracle   -                                         | pagerduty | papermill | plexus | postgres | presto | qubole | redis |         -                                         salesforce | samba | segment | sendgrid | sftp | singularity | slack |          -                                         snowflake | sqlite | ssh | tableau | telegram | trino | vertica | yandex |      -                                         zendesk]...                                                                     -                                                                                                                         - Prepare sdist/whl packages of Airflow Providers.                                                                        -                                                                                                                         -╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --package-format             Format of packages. (wheel | sdist | both) [default: wheel]                             -  --version-suffix-for-pypi    Version suffix used for PyPI packages (alpha, beta, rc1, etc.). (TEXT)                  -  --package-list-file          Read list of packages from text file (one package per line) (FILENAME)                  -  --debug                      Drop user in shell instead of running the command. Useful for debugging.                -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +                                                                                                                          + Usage: breeze prepare-provider-packages [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra |        +                                         apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin |        +                                         apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop |         +                                         arangodb | asana | celery | cloudant | cncf.kubernetes | databricks | datadog   +                                         | dbt.cloud | dingding | discord | docker | elasticsearch | exasol | facebook   +                                         | ftp | github | google | grpc | hashicorp | http | imap | influxdb | jdbc |    +                                         jenkins | jira | microsoft.azure | microsoft.mssql | microsoft.psrp |           +                                         microsoft.winrm | mongo | mysql | neo4j | odbc | openfaas | opsgenie | oracle   +                                         | pagerduty | papermill | plexus | postgres | presto | qubole | redis |         +                                         salesforce | samba | segment | sendgrid | sftp | singularity | slack |          +                                         snowflake | sqlite | ssh | tableau | tabular | telegram | trino | vertica |     +                                         yandex | zendesk]...                                                            +                                                                                                                         + Prepare sdist/whl packages of Airflow Providers.                                                                        +                                                                                                                         +╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ +  --package-format             Format of packages. (wheel | sdist | both) [default: wheel]                             +  --version-suffix-for-pypi    Version suffix used for PyPI packages (alpha, beta, rc1, etc.). (TEXT)                  +  --package-list-file          Read list of packages from text file (one package per line) (FILENAME)                  +  --debug                      Drop user in shell instead of running the command. Useful for debugging.                +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +  --verbose            -v  Print verbose information about performed steps.                                            +  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 +  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           +  --help               -h  Show this message and exit.                                                                 +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/setup.py b/setup.py index f5414fbafae05..56659c6148ca3 100644 --- a/setup.py +++ b/setup.py @@ -740,6 +740,7 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version 'sqlite': [], 'ssh': ssh, 'tableau': tableau, + 'tabular': [], 'telegram': telegram, 'trino': trino, 'vertica': vertica, diff --git a/tests/providers/tabular/__init__.py b/tests/providers/tabular/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/providers/tabular/__init__.py @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/tests/providers/tabular/hooks/__init__.py b/tests/providers/tabular/hooks/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/providers/tabular/hooks/__init__.py @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/tests/providers/tabular/hooks/test_tabular.py b/tests/providers/tabular/hooks/test_tabular.py new file mode 100644 index 0000000000000..b3c68452e5cfa --- /dev/null +++ b/tests/providers/tabular/hooks/test_tabular.py @@ -0,0 +1,42 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +import unittest + +import requests_mock + +from airflow.providers.tabular.hooks.tabular import TabularHook + + +class TestTabularHook(unittest.TestCase): + """Test endpoint""" + + def test_tabular(self): + access_token = 'eyJ0eXAiOiJKV1QiLCJhbGciOiJSU' + with requests_mock.Mocker() as m: + m.get( + 'https://api.tabulardata.io/ws/v1/oauth/tokens', + json={ + 'access_token': access_token, + 'token_type': 'Bearer', + 'expires_in': 86400, + 'warehouse_id': 'fadc4c31-e81f-48cd-9ce8-64cd5ce3fa5d', + 'region': 'us-west-2', + 'catalog_url': 'warehouses/fadc4c31-e81f-48cd-9ce8-64cd5ce3fa5d', + }, + ) + assert TabularHook().get_conn() == access_token From 4c4b8f774950691dfe9ce80ebe9987fb4b2042c5 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 24 May 2022 14:26:30 +0200 Subject: [PATCH 02/13] Github comments --- airflow/providers/tabular/example_dags/example_tabular.py | 2 +- airflow/providers/tabular/hooks/tabular.py | 8 ++++---- images/breeze/output-commands-hash.txt | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/airflow/providers/tabular/example_dags/example_tabular.py b/airflow/providers/tabular/example_dags/example_tabular.py index f72cfa4ed2b4d..b8088bf76c59a 100644 --- a/airflow/providers/tabular/example_dags/example_tabular.py +++ b/airflow/providers/tabular/example_dags/example_tabular.py @@ -38,7 +38,7 @@ }, start_date=datetime(2021, 1, 1), schedule_interval=timedelta(1), - catchup=False + catchup=False, ) as dag: # This also works for the SparkSubmit operator BashOperator( diff --git a/airflow/providers/tabular/hooks/tabular.py b/airflow/providers/tabular/hooks/tabular.py index 8ab92681aa362..11dbb877d6394 100644 --- a/airflow/providers/tabular/hooks/tabular.py +++ b/airflow/providers/tabular/hooks/tabular.py @@ -45,16 +45,16 @@ class TabularHook(BaseHook): def get_ui_field_behaviour() -> Dict[str, Any]: """Returns custom field behaviour""" return { - "hidden_fields": ["schema", "port", "host"], + "hidden_fields": ["schema", "port"], "relabeling": { "host": "Base URL", - "login": "Tabular Client ID", - "password": "Tabular Client Secret", + "login": "Client ID", + "password": "Client Secret", }, "placeholders": { "host": DEFAULT_TABULAR_URL, "login": "client_id (token credentials auth)", - "password": "secret (token credentials auth)" + "password": "secret (token credentials auth)", }, } diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt index f75b6cba57529..e206af365356b 100644 --- a/images/breeze/output-commands-hash.txt +++ b/images/breeze/output-commands-hash.txt @@ -1 +1 @@ -aa34cfc99f60649fea3b808ef225981e +55c0e8867b39c39e2dc448922dead842 From b5bb2d47d7ffdb361e3cc593dd1ba5dad32159ab Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 31 May 2022 21:24:39 +0200 Subject: [PATCH 03/13] Fix the docs --- images/breeze/output-build-docs.svg | 323 ++++++++++--- images/breeze/output-build-image.svg | 328 ++++++++++--- images/breeze/output-build-prod-image.svg | 418 ++++++++++++---- images/breeze/output-cleanup.svg | 118 +++-- images/breeze/output-command-hash-export.svg | 86 +++- images/breeze/output-commands-hash.txt | 2 +- images/breeze/output-commands.svg | 454 ++++++++++++++---- images/breeze/output-config.svg | 148 ++++-- images/breeze/output-docker-compose-tests.svg | 128 +++-- images/breeze/output-exec.svg | 91 ++-- .../breeze/output-find-newer-dependencies.svg | 133 +++-- images/breeze/output-fix-ownership.svg | 98 ++-- images/breeze/output-free-space.svg | 98 ++-- images/breeze/output-generate-constraints.svg | 188 ++++++-- .../breeze/output-prepare-airflow-package.svg | 123 +++-- .../output-prepare-provider-documentation.svg | 178 +++++-- .../output-prepare-provider-packages.svg | 183 +++++-- images/breeze/output-pull-image.svg | 183 +++++-- images/breeze/output-pull-prod-image.svg | 183 +++++-- images/breeze/output-release-prod-images.svg | 168 +++++-- images/breeze/output-resource-check.svg | 91 ++-- images/breeze/output-self-upgrade.svg | 101 ++-- images/breeze/output-setup-autocomplete.svg | 113 +++-- images/breeze/output-shell.svg | 264 +++++++--- images/breeze/output-start-airflow.svg | 274 ++++++++--- images/breeze/output-static-checks.svg | 285 ++++++++--- images/breeze/output-stop.svg | 106 ++-- images/breeze/output-tests.svg | 133 +++-- images/breeze/output-verify-image.svg | 128 +++-- images/breeze/output-verify-prod-image.svg | 128 +++-- .../output-verify-provider-packages.svg | 178 +++++-- images/breeze/output-version.svg | 86 +++- 32 files changed, 4120 insertions(+), 1398 deletions(-) diff --git a/images/breeze/output-build-docs.svg b/images/breeze/output-build-docs.svg index 4e01061e4f330..ab86dd44a54b7 100644 --- a/images/breeze/output-build-docs.svg +++ b/images/breeze/output-build-docs.svg @@ -1,4 +1,4 @@ - + - Command: build-docs - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: build-docs + + + + + - + -                                                                                                                          - Usage: breeze build-docs [OPTIONS]                                                                                      -                                                                                                                         - Build documentation in the container.                                                                                   -                                                                                                                         -╭─ Doc flags ──────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --docs-only        -d  Only build documentation.                                                                     -  --spellcheck-only  -s  Only run spell checking.                                                                      -  --for-production   -p  Builds documentation for official release i.e. all links point to stable version.             -  --package-filter   -p  List of packages to consider.                                                                 -                         (apache-airflow | apache-airflow-providers | apache-airflow-providers-airbyte |               -                         apache-airflow-providers-alibaba | apache-airflow-providers-amazon |                          -                         apache-airflow-providers-apache-beam | apache-airflow-providers-apache-cassandra |            -                         apache-airflow-providers-apache-drill | apache-airflow-providers-apache-druid |               -                         apache-airflow-providers-apache-hdfs | apache-airflow-providers-apache-hive |                 -                         apache-airflow-providers-apache-kylin | apache-airflow-providers-apache-livy |                -                         apache-airflow-providers-apache-pig | apache-airflow-providers-apache-pinot |                 -                         apache-airflow-providers-apache-spark | apache-airflow-providers-apache-sqoop |               -                         apache-airflow-providers-arangodb | apache-airflow-providers-asana |                          -                         apache-airflow-providers-celery | apache-airflow-providers-cloudant |                         -                         apache-airflow-providers-cncf-kubernetes | apache-airflow-providers-databricks |              -                         apache-airflow-providers-datadog | apache-airflow-providers-dbt-cloud |                       -                         apache-airflow-providers-dingding | apache-airflow-providers-discord |                        -                         apache-airflow-providers-docker | apache-airflow-providers-elasticsearch |                    -                         apache-airflow-providers-exasol | apache-airflow-providers-facebook |                         -                         apache-airflow-providers-ftp | apache-airflow-providers-github |                              -                         apache-airflow-providers-google | apache-airflow-providers-grpc |                             -                         apache-airflow-providers-hashicorp | apache-airflow-providers-http |                          -                         apache-airflow-providers-imap | apache-airflow-providers-influxdb |                           -                         apache-airflow-providers-jdbc | apache-airflow-providers-jenkins |                            -                         apache-airflow-providers-jira | apache-airflow-providers-microsoft-azure |                    -                         apache-airflow-providers-microsoft-mssql | apache-airflow-providers-microsoft-psrp |          -                         apache-airflow-providers-microsoft-winrm | apache-airflow-providers-mongo |                   -                         apache-airflow-providers-mysql | apache-airflow-providers-neo4j |                             -                         apache-airflow-providers-odbc | apache-airflow-providers-openfaas |                           -                         apache-airflow-providers-opsgenie | apache-airflow-providers-oracle |                         -                         apache-airflow-providers-pagerduty | apache-airflow-providers-papermill |                     -                         apache-airflow-providers-plexus | apache-airflow-providers-postgres |                         -                         apache-airflow-providers-presto | apache-airflow-providers-qubole |                           -                         apache-airflow-providers-redis | apache-airflow-providers-salesforce |                        -                         apache-airflow-providers-samba | apache-airflow-providers-segment |                           -                         apache-airflow-providers-sendgrid | apache-airflow-providers-sftp |                           -                         apache-airflow-providers-singularity | apache-airflow-providers-slack |                       -                         apache-airflow-providers-snowflake | apache-airflow-providers-sqlite |                        -                         apache-airflow-providers-ssh | apache-airflow-providers-tableau |                             -                         apache-airflow-providers-tabular | apache-airflow-providers-telegram |                        -                         apache-airflow-providers-trino | apache-airflow-providers-vertica |                           -                         apache-airflow-providers-yandex | apache-airflow-providers-zendesk | docker-stack |           -                         helm-chart)                                                                                   -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze build-docs [OPTIONS] + +Build documentation in the container. + +╭─ Doc flags ──────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--docs-only-dOnly build documentation. +--spellcheck-only-sOnly run spell checking. +--for-production-pBuilds documentation for official release i.e. all links point to stable version. +--package-filter-pList of packages to consider.                                                                 +(apache-airflow | apache-airflow-providers | apache-airflow-providers-airbyte |               +apache-airflow-providers-alibaba | apache-airflow-providers-amazon |                          +apache-airflow-providers-apache-beam | apache-airflow-providers-apache-cassandra |            +apache-airflow-providers-apache-drill | apache-airflow-providers-apache-druid |               +apache-airflow-providers-apache-hdfs | apache-airflow-providers-apache-hive |                 +apache-airflow-providers-apache-kylin | apache-airflow-providers-apache-livy |                +apache-airflow-providers-apache-pig | apache-airflow-providers-apache-pinot |                 +apache-airflow-providers-apache-spark | apache-airflow-providers-apache-sqoop |               +apache-airflow-providers-arangodb | apache-airflow-providers-asana |                          +apache-airflow-providers-celery | apache-airflow-providers-cloudant |                         +apache-airflow-providers-cncf-kubernetes | apache-airflow-providers-databricks |              +apache-airflow-providers-datadog | apache-airflow-providers-dbt-cloud |                       +apache-airflow-providers-dingding | apache-airflow-providers-discord |                        +apache-airflow-providers-docker | apache-airflow-providers-elasticsearch |                    +apache-airflow-providers-exasol | apache-airflow-providers-facebook |                         +apache-airflow-providers-ftp | apache-airflow-providers-github |                              +apache-airflow-providers-google | apache-airflow-providers-grpc |                             +apache-airflow-providers-hashicorp | apache-airflow-providers-http |                          +apache-airflow-providers-imap | apache-airflow-providers-influxdb |                           +apache-airflow-providers-jdbc | apache-airflow-providers-jenkins |                            +apache-airflow-providers-jira | apache-airflow-providers-microsoft-azure |                    +apache-airflow-providers-microsoft-mssql | apache-airflow-providers-microsoft-psrp |          +apache-airflow-providers-microsoft-winrm | apache-airflow-providers-mongo |                   +apache-airflow-providers-mysql | apache-airflow-providers-neo4j |                             +apache-airflow-providers-odbc | apache-airflow-providers-openfaas |                           +apache-airflow-providers-opsgenie | apache-airflow-providers-oracle |                         +apache-airflow-providers-pagerduty | apache-airflow-providers-papermill |                     +apache-airflow-providers-plexus | apache-airflow-providers-postgres |                         +apache-airflow-providers-presto | apache-airflow-providers-qubole |                           +apache-airflow-providers-redis | apache-airflow-providers-salesforce |                        +apache-airflow-providers-samba | apache-airflow-providers-segment |                           +apache-airflow-providers-sendgrid | apache-airflow-providers-sftp |                           +apache-airflow-providers-singularity | apache-airflow-providers-slack |                       +apache-airflow-providers-snowflake | apache-airflow-providers-sqlite |                        +apache-airflow-providers-ssh | apache-airflow-providers-tableau |                             +apache-airflow-providers-tabular | apache-airflow-providers-telegram |                        +apache-airflow-providers-trino | apache-airflow-providers-vertica |                           +apache-airflow-providers-yandex | apache-airflow-providers-zendesk | docker-stack |           +helm-chart)                                                                                   +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-build-image.svg b/images/breeze/output-build-image.svg index 6b555c27f3428..603f54c28c0e0 100644 --- a/images/breeze/output-build-image.svg +++ b/images/breeze/output-build-image.svg @@ -1,4 +1,4 @@ - + - Command: build-image - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: build-image + + + + + - + -                                                                                                                          - Usage: breeze build-image [OPTIONS]                                                                                     -                                                                                                                         - Build CI image. Include building multiple images for all python versions (sequentially).                                -                                                                                                                         -╭─ Basic usage ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --python                         -p  Python major/minor version used in Airflow image for images.                    -                                       (>3.7< | 3.8 | 3.9 | 3.10)                                                      -                                       [default: 3.7]                                                                  -  --upgrade-to-newer-dependencies  -u  When set, upgrade all PIP packages to latest.                                   -  --debian-version                     Debian version used for the image. (bullseye | buster) [default: bullseye]      -  --image-tag                      -t  Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT)   -  --tag-as-latest                      Tags the image as latest and update checksum of all files after pulling.        -                                       Useful when you build or pull image with --image-tag.                           -  --docker-cache                   -c  Cache option for image used during the build. (registry | local | disabled)     -                                       [default: registry]                                                             -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Building multiple images ───────────────────────────────────────────────────────────────────────────────────────────╮ -  --build-multiple-images    Run the operation sequentially on all or selected subset of Python versions.              -  --python-versions          Space separated list of python versions used for build with multiple versions. (TEXT)     -                             [default: 3.7 3.8 3.9 3.10]                                                               -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Advanced options (for power users) ─────────────────────────────────────────────────────────────────────────────────╮ -  --install-providers-from-sources    Install providers from sources when installing.                                  -  --airflow-constraints-mode          Mode of constraints for CI image building                                        -                                      (constraints-source-providers | constraints | constraints-no-providers)          -                                      [default: constraints-source-providers]                                          -  --airflow-constraints-reference     Constraint reference to use when building the image. (TEXT)                      -  --additional-python-deps            Additional python dependencies to use when building the images. (TEXT)           -  --runtime-apt-deps                  Apt runtime dependencies to use when building the images. (TEXT)                 -  --runtime-apt-command               Command executed before runtime apt deps are installed. (TEXT)                   -  --additional-extras                 Additional extra package while installing Airflow in the image. (TEXT)           -  --additional-runtime-apt-deps       Additional apt runtime dependencies to use when building the images. (TEXT)      -  --additional-runtime-apt-env        Additional environment variables set when adding runtime dependencies. (TEXT)    -  --additional-runtime-apt-command    Additional command executed before runtime apt deps are installed. (TEXT)        -  --additional-dev-apt-deps           Additional apt dev dependencies to use when building the images. (TEXT)          -  --additional-dev-apt-env            Additional environment variables set when adding dev dependencies. (TEXT)        -  --additional-dev-apt-command        Additional command executed before dev apt deps are installed. (TEXT)            -  --dev-apt-deps                      Apt dev dependencies to use when building the images. (TEXT)                     -  --dev-apt-command                   Command executed before dev apt deps are installed. (TEXT)                       -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Preparing cache and push (for maintainers and CI) ──────────────────────────────────────────────────────────────────╮ -  --github-token            The token used to authenticate to GitHub. (TEXT)                                           -  --github-username         The user name used to authenticate to GitHub. (TEXT)                                       -  --platform                Platform for Airflow image. (linux/amd64 | linux/arm64 | linux/amd64,linux/arm64)          -  --push-image              Push image after building it.                                                              -  --empty-image             Prepare empty image tagged with the same name as the Airflow image.                        -  --prepare-buildx-cache    Prepares build cache (this is done as separate per-platform steps instead of building      -                            the image).                                                                                -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --answer             -a  Force answer to questions. (y | n | q | yes | no | quit)                                    -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze build-image [OPTIONS] + +Build CI image. Include building multiple images for all python versions (sequentially). + +╭─ Basic usage ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images. +(>3.7< | 3.8 | 3.9 | 3.10)                                   +[default: 3.7]                                               +--upgrade-to-newer-dependencies-uWhen set, upgrade all PIP packages to latest. +--debian-versionDebian version used for the image.(bullseye | buster)[default: bullseye] +--image-tag-tTag added to the default naming conventions of Airflow CI/PROD images.(TEXT) +--tag-as-latestTags the image as latest and update checksum of all files after pulling. Useful +when you build or pull image with --image-tag.                                  +--docker-cache-cCache option for image used during the build.(registry | local | disabled) +[default: registry]                           +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Building multiple images ───────────────────────────────────────────────────────────────────────────────────────────╮ +--build-multiple-imagesRun the operation sequentially on all or selected subset of Python versions. +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Advanced options (for power users) ─────────────────────────────────────────────────────────────────────────────────╮ +--install-providers-from-sourcesInstall providers from sources when installing. +--airflow-constraints-modeMode of constraints for CI image building                               +(constraints-source-providers | constraints | constraints-no-providers) +[default: constraints-source-providers]                                 +--airflow-constraints-referenceConstraint reference to use when building the image.(TEXT) +--additional-python-depsAdditional python dependencies to use when building the images.(TEXT) +--runtime-apt-depsApt runtime dependencies to use when building the images.(TEXT) +--runtime-apt-commandCommand executed before runtime apt deps are installed.(TEXT) +--additional-extrasAdditional extra package while installing Airflow in the image.(TEXT) +--additional-runtime-apt-depsAdditional apt runtime dependencies to use when building the images.(TEXT) +--additional-runtime-apt-envAdditional environment variables set when adding runtime dependencies.(TEXT) +--additional-runtime-apt-commandAdditional command executed before runtime apt deps are installed.(TEXT) +--additional-dev-apt-depsAdditional apt dev dependencies to use when building the images.(TEXT) +--additional-dev-apt-envAdditional environment variables set when adding dev dependencies.(TEXT) +--additional-dev-apt-commandAdditional command executed before dev apt deps are installed.(TEXT) +--dev-apt-depsApt dev dependencies to use when building the images.(TEXT) +--dev-apt-commandCommand executed before dev apt deps are installed.(TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Preparing cache and push (for maintainers and CI) ──────────────────────────────────────────────────────────────────╮ +--github-tokenThe token used to authenticate to GitHub.(TEXT) +--github-usernameThe user name used to authenticate to GitHub.(TEXT) +--platformPlatform for Airflow image.(linux/amd64 | linux/arm64 | linux/amd64,linux/arm64) +--push-imagePush image after building it. +--empty-imagePrepare empty image tagged with the same name as the Airflow image. +--prepare-buildx-cachePrepares build cache (this is done as separate per-platform steps instead of building the  +image).                                                                                    +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-build-prod-image.svg b/images/breeze/output-build-prod-image.svg index 247461d55530c..494e77574c3e0 100644 --- a/images/breeze/output-build-prod-image.svg +++ b/images/breeze/output-build-prod-image.svg @@ -1,4 +1,4 @@ - + - Command: build-prod-image - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: build-prod-image + + + + + - + -                                                                                                                          - Usage: breeze build-prod-image [OPTIONS]                                                                                -                                                                                                                         - Build Production image. Include building multiple images for all or selected Python versions sequentially.              -                                                                                                                         -╭─ Basic usage ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --python                         -p  Python major/minor version used in Airflow image for images.                    -                                       (>3.7< | 3.8 | 3.9 | 3.10)                                                      -                                       [default: 3.7]                                                                  -  --install-airflow-version        -V  Install version of Airflow from PyPI. (TEXT)                                    -  --upgrade-to-newer-dependencies  -u  When set, upgrade all PIP packages to latest.                                   -  --debian-version                     Debian version used for the image. (bullseye | buster) [default: bullseye]      -  --image-tag                      -t  Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT)   -  --tag-as-latest                      Tags the image as latest and update checksum of all files after pulling.        -                                       Useful when you build or pull image with --image-tag.                           -  --docker-cache                   -c  Cache option for image used during the build. (registry | local | disabled)     -                                       [default: registry]                                                             -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Building multiple images ───────────────────────────────────────────────────────────────────────────────────────────╮ -  --build-multiple-images    Run the operation sequentially on all or selected subset of Python versions.              -  --python-versions          Space separated list of python versions used for build with multiple versions. (TEXT)     -                             [default: 3.7 3.8 3.9 3.10]                                                               -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options for customizing images ─────────────────────────────────────────────────────────────────────────────────────╮ -  --install-providers-from-sources    Install providers from sources when installing.                                  -  --airflow-extras                    Extras to install by default.                                                    -                                      (TEXT)                                                                           -                                      [default:                                                                        -                                      amazon,async,celery,cncf.kubernetes,dask,docker,elasticsearch,ftp,google,goog…   -  --airflow-constraints-mode          Mode of constraints for PROD image building                                      -                                      (constraints | constraints-no-providers | constraints-source-providers)          -                                      [default: constraints]                                                           -  --airflow-constraints-reference     Constraint reference to use when building the image. (TEXT)                      -  --additional-python-deps            Additional python dependencies to use when building the images. (TEXT)           -  --additional-extras                 Additional extra package while installing Airflow in the image. (TEXT)           -  --additional-runtime-apt-deps       Additional apt runtime dependencies to use when building the images. (TEXT)      -  --additional-runtime-apt-env        Additional environment variables set when adding runtime dependencies. (TEXT)    -  --additional-runtime-apt-command    Additional command executed before runtime apt deps are installed. (TEXT)        -  --additional-dev-apt-deps           Additional apt dev dependencies to use when building the images. (TEXT)          -  --additional-dev-apt-env            Additional environment variables set when adding dev dependencies. (TEXT)        -  --additional-dev-apt-command        Additional command executed before dev apt deps are installed. (TEXT)            -  --runtime-apt-deps                  Apt runtime dependencies to use when building the images. (TEXT)                 -  --runtime-apt-command               Command executed before runtime apt deps are installed. (TEXT)                   -  --dev-apt-deps                      Apt dev dependencies to use when building the images. (TEXT)                     -  --dev-apt-command                   Command executed before dev apt deps are installed. (TEXT)                       -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Customization options (for specific customization needs) ───────────────────────────────────────────────────────────╮ -  --install-packages-from-context           Install wheels from local docker-context-files when building image.        -  --airflow-is-in-context                   If set Airflow is installed from docker-context-files only rather than     -                                            from PyPI or sources.                                                      -  --cleanup-context                         Clean up docker context files before running build (cannot be used         -                                            together with --install-packages-from-context).                            -  --disable-mysql-client-installation       Do not install MySQL client.                                               -  --disable-mssql-client-installation       Do not install MsSQl client.                                               -  --disable-postgres-client-installation    Do not install Postgres client.                                            -  --disable-airflow-repo-cache              Disable cache from Airflow repository during building.                     -  --install-airflow-reference               Install Airflow using GitHub tag or branch. (TEXT)                         -  --installation-method                     Install Airflow from: sources or PyPI. (. | apache-airflow)                -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Preparing cache and push (for maintainers and CI) ──────────────────────────────────────────────────────────────────╮ -  --github-token            The token used to authenticate to GitHub. (TEXT)                                           -  --github-username         The user name used to authenticate to GitHub. (TEXT)                                       -  --platform                Platform for Airflow image. (linux/amd64 | linux/arm64 | linux/amd64,linux/arm64)          -  --push-image              Push image after building it.                                                              -  --empty-image             Prepare empty image tagged with the same name as the Airflow image.                        -  --prepare-buildx-cache    Prepares build cache (this is done as separate per-platform steps instead of building      -                            the image).                                                                                -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --answer             -a  Force answer to questions. (y | n | q | yes | no | quit)                                    -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --verbose            -v  Print verbose information about performed steps.                                            -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze build-prod-image [OPTIONS] + +Build Production image. Include building multiple images for all or selected Python versions sequentially. + +╭─ Basic usage ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images. +(>3.7< | 3.8 | 3.9 | 3.10)                                   +[default: 3.7]                                               +--install-airflow-version-VInstall version of Airflow from PyPI.(TEXT) +--upgrade-to-newer-dependencies-uWhen set, upgrade all PIP packages to latest. +--debian-versionDebian version used for the image.(bullseye | buster)[default: bullseye] +--image-tag-tTag added to the default naming conventions of Airflow CI/PROD images.(TEXT) +--tag-as-latestTags the image as latest and update checksum of all files after pulling. Useful +when you build or pull image with --image-tag.                                  +--docker-cache-cCache option for image used during the build.(registry | local | disabled) +[default: registry]                           +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Building multiple images ───────────────────────────────────────────────────────────────────────────────────────────╮ +--build-multiple-imagesRun the operation sequentially on all or selected subset of Python versions. +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options for customizing images ─────────────────────────────────────────────────────────────────────────────────────╮ +--install-providers-from-sourcesInstall providers from sources when installing. +--airflow-extrasExtras to install by default.                                                    +(TEXT)                                                                           +[default:                                                                        +amazon,async,celery,cncf.kubernetes,dask,docker,elasticsearch,ftp,google,google… +--airflow-constraints-modeMode of constraints for PROD image building                             +(constraints | constraints-no-providers | constraints-source-providers) +[default: constraints]                                                  +--airflow-constraints-referenceConstraint reference to use when building the image.(TEXT) +--additional-python-depsAdditional python dependencies to use when building the images.(TEXT) +--additional-extrasAdditional extra package while installing Airflow in the image.(TEXT) +--additional-runtime-apt-depsAdditional apt runtime dependencies to use when building the images.(TEXT) +--additional-runtime-apt-envAdditional environment variables set when adding runtime dependencies.(TEXT) +--additional-runtime-apt-commandAdditional command executed before runtime apt deps are installed.(TEXT) +--additional-dev-apt-depsAdditional apt dev dependencies to use when building the images.(TEXT) +--additional-dev-apt-envAdditional environment variables set when adding dev dependencies.(TEXT) +--additional-dev-apt-commandAdditional command executed before dev apt deps are installed.(TEXT) +--runtime-apt-depsApt runtime dependencies to use when building the images.(TEXT) +--runtime-apt-commandCommand executed before runtime apt deps are installed.(TEXT) +--dev-apt-depsApt dev dependencies to use when building the images.(TEXT) +--dev-apt-commandCommand executed before dev apt deps are installed.(TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Customization options (for specific customization needs) ───────────────────────────────────────────────────────────╮ +--install-packages-from-contextInstall wheels from local docker-context-files when building image. +--airflow-is-in-contextIf set Airflow is installed from docker-context-files only rather than     +from PyPI or sources.                                                      +--cleanup-contextClean up docker context files before running build (cannot be used         +together with --install-packages-from-context).                            +--disable-mysql-client-installationDo not install MySQL client. +--disable-mssql-client-installationDo not install MsSQl client. +--disable-postgres-client-installationDo not install Postgres client. +--disable-airflow-repo-cacheDisable cache from Airflow repository during building. +--install-airflow-referenceInstall Airflow using GitHub tag or branch.(TEXT) +--installation-methodInstall Airflow from: sources or PyPI.(. | apache-airflow) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Preparing cache and push (for maintainers and CI) ──────────────────────────────────────────────────────────────────╮ +--github-tokenThe token used to authenticate to GitHub.(TEXT) +--github-usernameThe user name used to authenticate to GitHub.(TEXT) +--platformPlatform for Airflow image.(linux/amd64 | linux/arm64 | linux/amd64,linux/arm64) +--push-imagePush image after building it. +--empty-imagePrepare empty image tagged with the same name as the Airflow image. +--prepare-buildx-cachePrepares build cache (this is done as separate per-platform steps instead of building the  +image).                                                                                    +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--verbose-vPrint verbose information about performed steps. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-cleanup.svg b/images/breeze/output-cleanup.svg index 68395e65ebe26..9bd373b98be83 100644 --- a/images/breeze/output-cleanup.svg +++ b/images/breeze/output-cleanup.svg @@ -1,4 +1,4 @@ - + - Command: cleanup - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: cleanup + + + + + - + -                                                                                                                          - Usage: breeze cleanup [OPTIONS]                                                                                         -                                                                                                                         - Cleans the cache of parameters, docker cache and optionally - currently downloaded images.                              -                                                                                                                         -╭─ Cleanup flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --all    Also remove currently downloaded Breeze images.                                                             -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --answer             -a  Force answer to questions. (y | n | q | yes | no | quit)                                    -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze cleanup [OPTIONS] + +Cleans the cache of parameters, docker cache and optionally - currently downloaded images. + +╭─ Cleanup flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--allAlso remove currently downloaded Breeze images. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-command-hash-export.svg b/images/breeze/output-command-hash-export.svg index 03d562e242f45..789317d72494f 100644 --- a/images/breeze/output-command-hash-export.svg +++ b/images/breeze/output-command-hash-export.svg @@ -1,4 +1,4 @@ - + - Command: command-hash-export - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: command-hash-export + + + + + - + -                                                                                                                          - Usage: breeze command-hash-export [OPTIONS] OUTPUT                                                                      -                                                                                                                         - Outputs hash of all click commands to file or stdout if `-` is used (useful to see if images should be regenerated).    -                                                                                                                         -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose  -v  Print verbose information about performed steps.                                                      -  --help     -h  Show this message and exit.                                                                           -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze command-hash-export [OPTIONS] OUTPUT + +Outputs hash of all click commands to file or stdout if `-` is used (useful to see if images should be regenerated). + +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt index e206af365356b..93bf455cad4d1 100644 --- a/images/breeze/output-commands-hash.txt +++ b/images/breeze/output-commands-hash.txt @@ -1 +1 @@ -55c0e8867b39c39e2dc448922dead842 +c6b08ab1a5811713da6b17c69f2157dc diff --git a/images/breeze/output-commands.svg b/images/breeze/output-commands.svg index 4fe025fcef29d..be4ea5a514d12 100644 --- a/images/breeze/output-commands.svg +++ b/images/breeze/output-commands.svg @@ -1,4 +1,4 @@ - + - Breeze commands - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Breeze commands + + + + + - + -                                                                                                                          - Usage: breeze [OPTIONS] COMMAND [ARGS]...                                                                               -                                                                                                                         -╭─ Basic flags for the default (shell) command ────────────────────────────────────────────────────────────────────────╮ -  --python               -p  Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10)   -                             [default: 3.7]                                                                            -  --backend              -b  Database backend to use. (>sqlite< | mysql | postgres | mssql) [default: sqlite]          -  --postgres-version     -P  Version of Postgres used. (>10< | 11 | 12 | 13 | 14) [default: 10]                        -  --mysql-version        -M  Version of MySQL used. (>5.7< | 8) [default: 5.7]                                         -  --mssql-version        -S  Version of MsSQL used. (>2017-latest< | 2019-latest) [default: 2017-latest]               -  --integration              Integration(s) to enable when running (can be more than one).                             -                             (cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino |    -                             all)                                                                                      -  --forward-credentials  -f  Forward local credentials to container when running.                                      -  --db-reset             -d  Reset DB when entering the container.                                                     -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Advanced flags for the default (shell) command ─────────────────────────────────────────────────────────────────────╮ -  --use-airflow-version       Use (reinstall at entry) Airflow version from PyPI. It can also be `none`, `wheel`, or   -                              `sdist` if Airflow should be removed, installed from wheel packages or sdist packages    -                              available in dist folder respectively. Implies --mount-sources `none`.                   -                              (none | wheel | sdist | <airflow_version>)                                               -  --airflow-extras            Airflow extras to install when --use-airflow-version is used (TEXT)                      -  --use-packages-from-dist    Install all found packages (--package-format determines type) from 'dist' folder when    -                              entering breeze.                                                                         -  --package-format            Format of packages that should be installed from dist. (wheel | sdist)                   -                              [default: wheel]                                                                         -  --force-build               Force image build no matter if it is determined as needed.                               -  --mount-sources             Choose scope of local sources should be mounted (default = selected).                    -                              (selected | all | none)                                                                  -                              [default: selected]                                                                      -  --debian-version            Debian version used for the image. (bullseye | buster) [default: bullseye]               -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --answer             -a  Force answer to questions. (y | n | q | yes | no | quit)                                    -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Developer tools ────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  shell          Enter breeze.py environment. this is the default command use when no other is selected.               -  start-airflow  Enter breeze.py environment and starts all Airflow components in the tmux session.                    -  exec           Joins the interactive shell of running airflow container                                              -  stop           Stop running breeze environment.                                                                      -  build-docs     Build documentation in the container.                                                                 -  static-checks  Run static checks.                                                                                    -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Testing ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  docker-compose-tests  Run docker-compose tests.                                                                      -  tests                 Run the specified unit test targets. Multiple targets may be specified separated by spaces.    -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Configuration & maintenance ────────────────────────────────────────────────────────────────────────────────────────╮ -  cleanup              Cleans the cache of parameters, docker cache and optionally - currently downloaded images.      -  self-upgrade         Self upgrade Breeze.                                                                            -  setup-autocomplete   Enables autocompletion of breeze commands.                                                      -  config               Show/update configuration (Python, Backend, Cheatsheet, ASCIIART).                              -  resource-check       Check if available docker resources are enough.                                                 -  free-space           Free space for jobs run in CI.                                                                  -  fix-ownership        Fix ownership of source files to be same as host user.                                          -  command-hash-export  Outputs hash of all click commands to file or stdout if `-` is used (useful to see if images    -                       should be regenerated).                                                                         -  version              Print information about version of apache-airflow-breeze.                                       -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ CI Image tools ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  build-image   Build CI image. Include building multiple images for all python versions (sequentially).               -  pull-image    Pull and optionally verify CI images - possibly in parallel for all Python versions.                   -  verify-image  Verify CI image.                                                                                       -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Production Image tools ─────────────────────────────────────────────────────────────────────────────────────────────╮ -  build-prod-image   Build Production image. Include building multiple images for all or selected Python versions      -                     sequentially.                                                                                     -  pull-prod-image    Pull and optionally verify Production images - possibly in parallel for all Python versions.      -  verify-prod-image  Verify Production image.                                                                          -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Release management ─────────────────────────────────────────────────────────────────────────────────────────────────╮ -  verify-provider-packages        Verifies if all provider code is following expectations for providers.               -  prepare-provider-documentation  Prepare CHANGELOG, README and COMMITS information for providers.                     -  prepare-provider-packages       Prepare sdist/whl packages of Airflow Providers.                                     -  prepare-airflow-package         Prepare sdist/whl package of Airflow.                                                -  release-prod-images             Release production images to DockerHub (needs DockerHub permissions).                -  generate-constraints            Generates pinned constraint files with all extras from setup.py in parallel.         -  find-newer-dependencies         Finds which dependencies are being upgraded.                                         -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze [OPTIONS] COMMAND [ARGS]... + +╭─ Basic flags for the default (shell) command ────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] +--postgres-version-PVersion of Postgres used.(>10< | 11 | 12 | 13 | 14)[default: 10] +--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] +--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] +--integrationIntegration(s) to enable when running (can be more than one).                             +(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino |    +all)                                                                                      +--forward-credentials-fForward local credentials to container when running. +--db-reset-dReset DB when entering the container. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Advanced flags for the default (shell) command ─────────────────────────────────────────────────────────────────────╮ +--use-airflow-versionUse (reinstall at entry) Airflow version from PyPI. It can also be `none`, `wheel`, or   +`sdist` if Airflow should be removed, installed from wheel packages or sdist packages    +available in dist folder respectively. Implies --mount-sources `none`.                   +(none | wheel | sdist | <airflow_version>)                                               +--airflow-extrasAirflow extras to install when --use-airflow-version is used(TEXT) +--use-packages-from-distInstall all found packages (--package-format determines type) from 'dist' folder when    +entering breeze.                                                                         +--package-formatFormat of packages that should be installed from dist.(wheel | sdist)[default: wheel] +--force-buildForce image build no matter if it is determined as needed. +--mount-sourcesChoose scope of local sources should be mounted (default = selected). +(selected | all | none)                                               +[default: selected]                                                   +--debian-versionDebian version used for the image.(bullseye | buster)[default: bullseye] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Developer tools ────────────────────────────────────────────────────────────────────────────────────────────────────╮ +shell          Enter breeze.py environment. this is the default command use when no other is selected.             +start-airflow  Enter breeze.py environment and starts all Airflow components in the tmux session.                  +exec           Joins the interactive shell of running airflow container                                            +stop           Stop running breeze environment.                                                                    +build-docs     Build documentation in the container.                                                               +static-checks  Run static checks.                                                                                  +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Testing ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +docker-compose-tests Run docker-compose tests.                                                                     +tests                Run the specified unit test targets. Multiple targets may be specified separated by spaces.   +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Configuration & maintenance ────────────────────────────────────────────────────────────────────────────────────────╮ +cleanup            Cleans the cache of parameters, docker cache and optionally - currently downloaded images.      +self-upgrade       Self upgrade Breeze.                                                                            +setup-autocomplete Enables autocompletion of breeze commands.                                                      +config             Show/update configuration (Python, Backend, Cheatsheet, ASCIIART).                              +resource-check     Check if available docker resources are enough.                                                 +free-space         Free space for jobs run in CI.                                                                  +fix-ownership      Fix ownership of source files to be same as host user.                                          +command-hash-exportOutputs hash of all click commands to file or stdout if `-` is used (useful to see if images    +should be regenerated).                                                                         +version            Print information about version of apache-airflow-breeze.                                       +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ CI Image tools ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +build-image   Build CI image. Include building multiple images for all python versions (sequentially).             +pull-image    Pull and optionally verify CI images - possibly in parallel for all Python versions.                 +verify-image  Verify CI image.                                                                                     +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Production Image tools ─────────────────────────────────────────────────────────────────────────────────────────────╮ +build-prod-image Build Production image. Include building multiple images for all or selected Python versions      +sequentially.                                                                                     +pull-prod-image  Pull and optionally verify Production images - possibly in parallel for all Python versions.      +verify-prod-imageVerify Production image.                                                                          +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Release management ─────────────────────────────────────────────────────────────────────────────────────────────────╮ +verify-provider-packages         Verifies if all provider code is following expectations for providers.            +prepare-provider-documentation   Prepare CHANGELOG, README and COMMITS information for providers.                  +prepare-provider-packages        Prepare sdist/whl packages of Airflow Providers.                                  +prepare-airflow-package          Prepare sdist/whl package of Airflow.                                             +release-prod-images              Release production images to DockerHub (needs DockerHub permissions).             +generate-constraints             Generates pinned constraint files with all extras from setup.py in parallel.      +find-newer-dependencies          Finds which dependencies are being upgraded.                                      +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-config.svg b/images/breeze/output-config.svg index f3c2e8c23e987..b196ab3306b24 100644 --- a/images/breeze/output-config.svg +++ b/images/breeze/output-config.svg @@ -1,4 +1,4 @@ - + - Command: config - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: config + + + + + - + -                                                                                                                          - Usage: breeze config [OPTIONS]                                                                                          -                                                                                                                         - Show/update configuration (Python, Backend, Cheatsheet, ASCIIART).                                                      -                                                                                                                         -╭─ Config flags ───────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --python                      -p     Python major/minor version used in Airflow image for images.                    -                                       (>3.7< | 3.8 | 3.9 | 3.10)                                                      -                                       [default: 3.7]                                                                  -  --backend                     -b     Database backend to use. (>sqlite< | mysql | postgres | mssql)                  -                                       [default: sqlite]                                                               -  --cheatsheet/--no-cheatsheet  -C/-c  Enable/disable cheatsheet.                                                      -  --asciiart/--no-asciiart      -A/-a  Enable/disable ASCIIart.                                                        -  --colour/--no-colour          -B/-b  Enable/disable Colour mode (useful for colour blind-friendly communication).    -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --postgres-version  -P  Version of Postgres used. (>10< | 11 | 12 | 13 | 14) [default: 10]                           -  --mysql-version     -M  Version of MySQL used. (>5.7< | 8) [default: 5.7]                                            -  --mssql-version     -S  Version of MsSQL used. (>2017-latest< | 2019-latest) [default: 2017-latest]                  -  --help              -h  Show this message and exit.                                                                  -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze config [OPTIONS] + +Show/update configuration (Python, Backend, Cheatsheet, ASCIIART). + +╭─ Config flags ───────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images. +(>3.7< | 3.8 | 3.9 | 3.10)                                   +[default: 3.7]                                               +--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql) +[default: sqlite]        +--cheatsheet/--no-cheatsheet-C/-cEnable/disable cheatsheet. +--asciiart/--no-asciiart-A/-aEnable/disable ASCIIart. +--colour/--no-colour-B/-bEnable/disable Colour mode (useful for colour blind-friendly communication). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--postgres-version-PVersion of Postgres used.(>10< | 11 | 12 | 13 | 14)[default: 10] +--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] +--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-docker-compose-tests.svg b/images/breeze/output-docker-compose-tests.svg index 71d16ecda4421..4830ca1215289 100644 --- a/images/breeze/output-docker-compose-tests.svg +++ b/images/breeze/output-docker-compose-tests.svg @@ -1,4 +1,4 @@ - + - Command: docker-compose-tests - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: docker-compose-tests + + + + + - + -                                                                                                                          - Usage: breeze docker-compose-tests [OPTIONS] [EXTRA_PYTEST_ARGS]...                                                     -                                                                                                                         - Run docker-compose tests.                                                                                               -                                                                                                                         -╭─ Docker-compose tests flag ──────────────────────────────────────────────────────────────────────────────────────────╮ -  --image-name  -n  Name of the image to verify (overrides --python and --image-tag). (TEXT)                           -  --python      -p  Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10)            -                    [default: 3.7]                                                                                     -  --image-tag   -t  Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT)                      -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze docker-compose-tests [OPTIONS] [EXTRA_PYTEST_ARGS]... + +Run docker-compose tests. + +╭─ Docker-compose tests flag ──────────────────────────────────────────────────────────────────────────────────────────╮ +--image-name-nName of the image to verify (overrides --python and --image-tag).(TEXT) +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--image-tag-tTag added to the default naming conventions of Airflow CI/PROD images.(TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-exec.svg b/images/breeze/output-exec.svg index b197eb3d63d1b..b8e69cd3addef 100644 --- a/images/breeze/output-exec.svg +++ b/images/breeze/output-exec.svg @@ -1,4 +1,4 @@ - + - Command: exec - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: exec + + + + + - + -                                                                                                                          - Usage: breeze exec [OPTIONS] [EXEC_ARGS]...                                                                             -                                                                                                                         - Joins the interactive shell of running airflow container                                                                -                                                                                                                         -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose  -v  Print verbose information about performed steps.                                                      -  --dry-run  -D  If dry-run is set, commands are only printed, not executed.                                           -  --help     -h  Show this message and exit.                                                                           -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze exec [OPTIONS] [EXEC_ARGS]... + +Joins the interactive shell of running airflow container + +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-find-newer-dependencies.svg b/images/breeze/output-find-newer-dependencies.svg index ea60bb6ab5cc9..43fc632ead7ed 100644 --- a/images/breeze/output-find-newer-dependencies.svg +++ b/images/breeze/output-find-newer-dependencies.svg @@ -1,4 +1,4 @@ - + - Command: find-newer-dependencies - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: find-newer-dependencies + + + + + - + -                                                                                                                          - Usage: breeze find-newer-dependencies [OPTIONS]                                                                         -                                                                                                                         - Finds which dependencies are being upgraded.                                                                            -                                                                                                                         -╭─ Find newer dependencies flags ──────────────────────────────────────────────────────────────────────────────────────╮ -  --python               -p  Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10)   -                             [default: 3.7]                                                                            -  --timezone                 Timezone to use during the check (TEXT)                                                   -  --updated-on-or-after      Date when the release was updated after (TEXT)                                            -  --max-age                  Max age of the last release (used if no updated-on-or-after if specified) (INTEGER)       -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --airflow-constraints-reference      Constraint reference to use. Useful with --use-airflow-version parameter to     -                                       specify constraints for the installed version and to find newer dependencies    -                                       (TEXT)                                                                          -  --help                           -h  Show this message and exit.                                                     -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze find-newer-dependencies [OPTIONS] + +Finds which dependencies are being upgraded. + +╭─ Find newer dependencies flags ──────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--timezoneTimezone to use during the check(TEXT) +--updated-on-or-afterDate when the release was updated after(TEXT) +--max-ageMax age of the last release (used if no updated-on-or-after if specified)(INTEGER) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--airflow-constraints-referenceConstraint reference to use. Useful with --use-airflow-version parameter to     +specify constraints for the installed version and to find newer dependencies    +(TEXT)                                                                          +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-fix-ownership.svg b/images/breeze/output-fix-ownership.svg index a35f337ea38e0..d8fe2ea84f440 100644 --- a/images/breeze/output-fix-ownership.svg +++ b/images/breeze/output-fix-ownership.svg @@ -1,4 +1,4 @@ - + - Command: fix-ownership - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: fix-ownership + + + + + - + -                                                                                                                          - Usage: breeze fix-ownership [OPTIONS]                                                                                   -                                                                                                                         - Fix ownership of source files to be same as host user.                                                                  -                                                                                                                         -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze fix-ownership [OPTIONS] + +Fix ownership of source files to be same as host user. + +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-free-space.svg b/images/breeze/output-free-space.svg index d0dd2934fe5fe..e0e4296201385 100644 --- a/images/breeze/output-free-space.svg +++ b/images/breeze/output-free-space.svg @@ -1,4 +1,4 @@ - + - Command: free-space - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: free-space + + + + + - + -                                                                                                                          - Usage: breeze free-space [OPTIONS]                                                                                      -                                                                                                                         - Free space for jobs run in CI.                                                                                          -                                                                                                                         -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose  -v  Print verbose information about performed steps.                                                      -  --dry-run  -D  If dry-run is set, commands are only printed, not executed.                                           -  --answer   -a  Force answer to questions. (y | n | q | yes | no | quit)                                              -  --help     -h  Show this message and exit.                                                                           -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze free-space [OPTIONS] + +Free space for jobs run in CI. + +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-generate-constraints.svg b/images/breeze/output-generate-constraints.svg index ac2afa05a16bb..ca049566070eb 100644 --- a/images/breeze/output-generate-constraints.svg +++ b/images/breeze/output-generate-constraints.svg @@ -1,4 +1,4 @@ - + - Command: generate-constraints - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: generate-constraints + + + + + - + -                                                                                                                          - Usage: breeze generate-constraints [OPTIONS]                                                                            -                                                                                                                         - Generates pinned constraint files with all extras from setup.py in parallel.                                            -                                                                                                                         -╭─ Generate constraints flags ─────────────────────────────────────────────────────────────────────────────────────────╮ -  --image-tag                 -t  Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT)        -  --python                    -p  Python major/minor version used in Airflow image for images.                         -                                  (>3.7< | 3.8 | 3.9 | 3.10)                                                           -                                  [default: 3.7]                                                                       -  --airflow-constraints-mode      Mode of constraints for CI image building                                            -                                  (constraints-source-providers | constraints | constraints-no-providers)              -                                  [default: constraints-source-providers]                                              -  --debug                         Drop user in shell instead of running the command. Useful for debugging.             -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --run-in-parallel    Run the operation in parallel on all or selected subset of Python versions.                     -  --parallelism        Maximum number of processes to use while running the operation in parallel. (INTEGER RANGE)     -                       [default: 4; 1<=x<=8]                                                                           -  --python-versions    Space separated list of python versions used for build with multiple versions. (TEXT)           -                       [default: 3.7 3.8 3.9 3.10]                                                                     -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --answer             -a  Force answer to questions. (y | n | q | yes | no | quit)                                    -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze generate-constraints [OPTIONS] + +Generates pinned constraint files with all extras from setup.py in parallel. + +╭─ Generate constraints flags ─────────────────────────────────────────────────────────────────────────────────────────╮ +--image-tag-tTag added to the default naming conventions of Airflow CI/PROD images.(TEXT) +--python-pPython major/minor version used in Airflow image for images. +(>3.7< | 3.8 | 3.9 | 3.10)                                   +[default: 3.7]                                               +--airflow-constraints-modeMode of constraints for CI image building                               +(constraints-source-providers | constraints | constraints-no-providers) +[default: constraints-source-providers]                                 +--debugDrop user in shell instead of running the command. Useful for debugging. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel.(INTEGER RANGE) +[default: 4; 1<=x<=8]                                                       +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-prepare-airflow-package.svg b/images/breeze/output-prepare-airflow-package.svg index 2a3c147b511fb..996151f94cc6d 100644 --- a/images/breeze/output-prepare-airflow-package.svg +++ b/images/breeze/output-prepare-airflow-package.svg @@ -1,4 +1,4 @@ - + - Command: prepare-airflow-package - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: prepare-airflow-package + + + + + - + -                                                                                                                          - Usage: breeze prepare-airflow-package [OPTIONS]                                                                         -                                                                                                                         - Prepare sdist/whl package of Airflow.                                                                                   -                                                                                                                         -╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --package-format             Format of packages. (wheel | sdist | both) [default: wheel]                             -  --version-suffix-for-pypi    Version suffix used for PyPI packages (alpha, beta, rc1, etc.). (TEXT)                  -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --debug                  Drop user in shell instead of running the command. Useful for debugging.                    -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze prepare-airflow-package [OPTIONS] + +Prepare sdist/whl package of Airflow. + +╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--package-formatFormat of packages.(wheel | sdist | both)[default: wheel] +--version-suffix-for-pypiVersion suffix used for PyPI packages (alpha, beta, rc1, etc.).(TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--debugDrop user in shell instead of running the command. Useful for debugging. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-prepare-provider-documentation.svg b/images/breeze/output-prepare-provider-documentation.svg index 47138dc074c4f..f0f660bf5d42b 100644 --- a/images/breeze/output-prepare-provider-documentation.svg +++ b/images/breeze/output-prepare-provider-documentation.svg @@ -1,4 +1,4 @@ - + - Command: prepare-provider-documentation - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: prepare-provider-documentation + + + + + - + -                                                                                                                          - Usage: breeze prepare-provider-documentation [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra |   -                                              apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin |   -                                              apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop |    -                                              arangodb | asana | celery | cloudant | cncf.kubernetes | databricks |      -                                              datadog | dbt.cloud | dingding | discord | docker | elasticsearch |        -                                              exasol | facebook | ftp | github | google | grpc | hashicorp | http |      -                                              imap | influxdb | jdbc | jenkins | jira | microsoft.azure |                -                                              microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql |       -                                              neo4j | odbc | openfaas | opsgenie | oracle | pagerduty | papermill |      -                                              plexus | postgres | presto | qubole | redis | salesforce | samba |         -                                              segment | sendgrid | sftp | singularity | slack | snowflake | sqlite |     -                                              ssh | tableau | tabular | telegram | trino | vertica | yandex |            -                                              zendesk]...                                                                -                                                                                                                         - Prepare CHANGELOG, README and COMMITS information for providers.                                                        -                                                                                                                         -╭─ Provider documentation preparation flags ───────────────────────────────────────────────────────────────────────────╮ -  --debug    Drop user in shell instead of running the command. Useful for debugging.                                  -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --answer             -a  Force answer to questions. (y | n | q | yes | no | quit)                                    -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze prepare-provider-documentation [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra | +                                             apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin | +                                             apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop | +                                             arangodb | asana | celery | cloudant | cncf.kubernetes | databricks | +                                             datadog | dbt.cloud | dingding | discord | docker | elasticsearch | +                                             exasol | facebook | ftp | github | google | grpc | hashicorp | http | +                                             imap | influxdb | jdbc | jenkins | jira | microsoft.azure | +                                             microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql | +                                             neo4j | odbc | openfaas | opsgenie | oracle | pagerduty | papermill | +                                             plexus | postgres | presto | qubole | redis | salesforce | samba | +                                             segment | sendgrid | sftp | singularity | slack | snowflake | sqlite | +                                             ssh | tableau | tabular | telegram | trino | vertica | yandex | +                                             zendesk]... + +Prepare CHANGELOG, README and COMMITS information for providers. + +╭─ Provider documentation preparation flags ───────────────────────────────────────────────────────────────────────────╮ +--debugDrop user in shell instead of running the command. Useful for debugging. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-prepare-provider-packages.svg b/images/breeze/output-prepare-provider-packages.svg index 5d0f70952a1c5..091257db2bd9a 100644 --- a/images/breeze/output-prepare-provider-packages.svg +++ b/images/breeze/output-prepare-provider-packages.svg @@ -1,4 +1,4 @@ - + - Command: prepare-provider-packages - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: prepare-provider-packages + + + + + - + -                                                                                                                          - Usage: breeze prepare-provider-packages [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra |        -                                         apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin |        -                                         apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop |         -                                         arangodb | asana | celery | cloudant | cncf.kubernetes | databricks | datadog   -                                         | dbt.cloud | dingding | discord | docker | elasticsearch | exasol | facebook   -                                         | ftp | github | google | grpc | hashicorp | http | imap | influxdb | jdbc |    -                                         jenkins | jira | microsoft.azure | microsoft.mssql | microsoft.psrp |           -                                         microsoft.winrm | mongo | mysql | neo4j | odbc | openfaas | opsgenie | oracle   -                                         | pagerduty | papermill | plexus | postgres | presto | qubole | redis |         -                                         salesforce | samba | segment | sendgrid | sftp | singularity | slack |          -                                         snowflake | sqlite | ssh | tableau | tabular | telegram | trino | vertica |     -                                         yandex | zendesk]...                                                            -                                                                                                                         - Prepare sdist/whl packages of Airflow Providers.                                                                        -                                                                                                                         -╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --package-format             Format of packages. (wheel | sdist | both) [default: wheel]                             -  --version-suffix-for-pypi    Version suffix used for PyPI packages (alpha, beta, rc1, etc.). (TEXT)                  -  --package-list-file          Read list of packages from text file (one package per line) (FILENAME)                  -  --debug                      Drop user in shell instead of running the command. Useful for debugging.                -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze prepare-provider-packages [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra | +                                        apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin | +                                        apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop | +                                        arangodb | asana | celery | cloudant | cncf.kubernetes | databricks | datadog +                                        | dbt.cloud | dingding | discord | docker | elasticsearch | exasol | facebook +                                        | ftp | github | google | grpc | hashicorp | http | imap | influxdb | jdbc | +                                        jenkins | jira | microsoft.azure | microsoft.mssql | microsoft.psrp | +                                        microsoft.winrm | mongo | mysql | neo4j | odbc | openfaas | opsgenie | oracle +                                        | pagerduty | papermill | plexus | postgres | presto | qubole | redis | +                                        salesforce | samba | segment | sendgrid | sftp | singularity | slack | +                                        snowflake | sqlite | ssh | tableau | tabular | telegram | trino | vertica | +                                        yandex | zendesk]... + +Prepare sdist/whl packages of Airflow Providers. + +╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--package-formatFormat of packages.(wheel | sdist | both)[default: wheel] +--version-suffix-for-pypiVersion suffix used for PyPI packages (alpha, beta, rc1, etc.).(TEXT) +--package-list-fileRead list of packages from text file (one package per line)(FILENAME) +--debugDrop user in shell instead of running the command. Useful for debugging. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-pull-image.svg b/images/breeze/output-pull-image.svg index 42cf6b6689f44..e1cce7beebc1c 100644 --- a/images/breeze/output-pull-image.svg +++ b/images/breeze/output-pull-image.svg @@ -1,4 +1,4 @@ - + - Command: pull-image - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: pull-image + + + + + - + -                                                                                                                          - Usage: breeze pull-image [OPTIONS] [EXTRA_PYTEST_ARGS]...                                                               -                                                                                                                         - Pull and optionally verify CI images - possibly in parallel for all Python versions.                                    -                                                                                                                         -╭─ Pull image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --image-tag       -t  Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT)                  -  --python          -p  Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10)        -                        [default: 3.7]                                                                                 -  --github-token        The token used to authenticate to GitHub. (TEXT)                                               -  --verify-image        Verify image.                                                                                  -  --wait-for-image      Wait until image is available.                                                                 -  --tag-as-latest       Tags the image as latest and update checksum of all files after pulling. Useful when you       -                        build or pull image with --image-tag.                                                          -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --run-in-parallel    Run the operation in parallel on all or selected subset of Python versions.                     -  --parallelism        Maximum number of processes to use while running the operation in parallel. (INTEGER RANGE)     -                       [default: 4; 1<=x<=8]                                                                           -  --python-versions    Space separated list of python versions used for build with multiple versions. (TEXT)           -                       [default: 3.7 3.8 3.9 3.10]                                                                     -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze pull-image [OPTIONS] [EXTRA_PYTEST_ARGS]... + +Pull and optionally verify CI images - possibly in parallel for all Python versions. + +╭─ Pull image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--image-tag-tTag added to the default naming conventions of Airflow CI/PROD images.(TEXT) +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--github-tokenThe token used to authenticate to GitHub.(TEXT) +--verify-imageVerify image. +--wait-for-imageWait until image is available. +--tag-as-latestTags the image as latest and update checksum of all files after pulling. Useful when you build +or pull image with --image-tag.                                                                +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel.(INTEGER RANGE) +[default: 4; 1<=x<=8]                                                       +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-pull-prod-image.svg b/images/breeze/output-pull-prod-image.svg index 9a12580b3bea1..d6ee293d03b8a 100644 --- a/images/breeze/output-pull-prod-image.svg +++ b/images/breeze/output-pull-prod-image.svg @@ -1,4 +1,4 @@ - + - Command: pull-prod-image - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: pull-prod-image + + + + + - + -                                                                                                                          - Usage: breeze pull-prod-image [OPTIONS] [EXTRA_PYTEST_ARGS]...                                                          -                                                                                                                         - Pull and optionally verify Production images - possibly in parallel for all Python versions.                            -                                                                                                                         -╭─ Pull image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --image-tag       -t  Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT)                  -  --python          -p  Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10)        -                        [default: 3.7]                                                                                 -  --github-token        The token used to authenticate to GitHub. (TEXT)                                               -  --verify-image        Verify image.                                                                                  -  --wait-for-image      Wait until image is available.                                                                 -  --tag-as-latest       Tags the image as latest and update checksum of all files after pulling. Useful when you       -                        build or pull image with --image-tag.                                                          -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --run-in-parallel    Run the operation in parallel on all or selected subset of Python versions.                     -  --parallelism        Maximum number of processes to use while running the operation in parallel. (INTEGER RANGE)     -                       [default: 4; 1<=x<=8]                                                                           -  --python-versions    Space separated list of python versions used for build with multiple versions. (TEXT)           -                       [default: 3.7 3.8 3.9 3.10]                                                                     -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze pull-prod-image [OPTIONS] [EXTRA_PYTEST_ARGS]... + +Pull and optionally verify Production images - possibly in parallel for all Python versions. + +╭─ Pull image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--image-tag-tTag added to the default naming conventions of Airflow CI/PROD images.(TEXT) +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--github-tokenThe token used to authenticate to GitHub.(TEXT) +--verify-imageVerify image. +--wait-for-imageWait until image is available. +--tag-as-latestTags the image as latest and update checksum of all files after pulling. Useful when you build +or pull image with --image-tag.                                                                +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel.(INTEGER RANGE) +[default: 4; 1<=x<=8]                                                       +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-release-prod-images.svg b/images/breeze/output-release-prod-images.svg index 31f4da7b4d847..a3a392acc5583 100644 --- a/images/breeze/output-release-prod-images.svg +++ b/images/breeze/output-release-prod-images.svg @@ -1,4 +1,4 @@ - + - Command: release-prod-images - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: release-prod-images + + + + + - + -                                                                                                                          - Usage: breeze release-prod-images [OPTIONS]                                                                             -                                                                                                                         - Release production images to DockerHub (needs DockerHub permissions).                                                   -                                                                                                                         -╭─ Release PROD IMAGE flags ───────────────────────────────────────────────────────────────────────────────────────────╮ -  *  --airflow-version    Airflow version to release (2.3.0, 2.3.0rc1 etc.) (TEXT) [required]                          -     --dockerhub-repo     DockerHub repository for the images (TEXT) [default: apache/airflow]                         -     --slim-images        Whether to prepare slim images instead of the regular ones.                                  -     --limit-python       Specific python to build slim images for (if not specified - the images are built for all    -                          available python versions)                                                                   -                          (3.7 | 3.8 | 3.9 | 3.10)                                                                     -     --limit-platform     Specific platform to build images for (if not specified, multiplatform images will be        -                          built.                                                                                       -                          (linux/amd64 | linux/arm64 | linux/amd64,linux/arm64)                                        -                          [default: linux/amd64,linux/arm64]                                                           -     --skip-latest        Whether to skip publishing the latest images (so that 'latest' images are not updated).      -                          This should only be used if you release image for previous branches. Automatically set       -                          when rc/alpha/beta images are built.                                                         -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose  -v  Print verbose information about performed steps.                                                      -  --dry-run  -D  If dry-run is set, commands are only printed, not executed.                                           -  --help     -h  Show this message and exit.                                                                           -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze release-prod-images [OPTIONS] + +Release production images to DockerHub (needs DockerHub permissions). + +╭─ Release PROD IMAGE flags ───────────────────────────────────────────────────────────────────────────────────────────╮ +*--airflow-versionAirflow version to release (2.3.0, 2.3.0rc1 etc.)(TEXT)[required] +--dockerhub-repoDockerHub repository for the images(TEXT)[default: apache/airflow] +--slim-imagesWhether to prepare slim images instead of the regular ones. +--limit-pythonSpecific python to build slim images for (if not specified - the images are built for all    +available python versions)                                                                   +(3.7 | 3.8 | 3.9 | 3.10)                                                                     +--limit-platformSpecific platform to build images for (if not specified, multiplatform images will be built. +(linux/amd64 | linux/arm64 | linux/amd64,linux/arm64)                                        +[default: linux/amd64,linux/arm64]                                                           +--skip-latestWhether to skip publishing the latest images (so that 'latest' images are not updated). This +should only be used if you release image for previous branches. Automatically set when       +rc/alpha/beta images are built.                                                              +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-resource-check.svg b/images/breeze/output-resource-check.svg index 310eaf9a1e707..8ec8aa32d7e7c 100644 --- a/images/breeze/output-resource-check.svg +++ b/images/breeze/output-resource-check.svg @@ -1,4 +1,4 @@ - + - Command: resource-check - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: resource-check + + + + + - + -                                                                                                                          - Usage: breeze resource-check [OPTIONS]                                                                                  -                                                                                                                         - Check if available docker resources are enough.                                                                         -                                                                                                                         -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose  -v  Print verbose information about performed steps.                                                      -  --dry-run  -D  If dry-run is set, commands are only printed, not executed.                                           -  --help     -h  Show this message and exit.                                                                           -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze resource-check [OPTIONS] + +Check if available docker resources are enough. + +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-self-upgrade.svg b/images/breeze/output-self-upgrade.svg index 147e6581b2598..205b6f1042e64 100644 --- a/images/breeze/output-self-upgrade.svg +++ b/images/breeze/output-self-upgrade.svg @@ -1,4 +1,4 @@ - + - Command: self-upgrade - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: self-upgrade + + + + + - + -                                                                                                                          - Usage: breeze self-upgrade [OPTIONS]                                                                                    -                                                                                                                         - Self upgrade Breeze.                                                                                                    -                                                                                                                         -╭─ Self-upgrade flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --use-current-airflow-sources  -a  Use current workdir Airflow sources for upgrade.                                  -  --force                        -f  Force upgrade without asking question to the user.                                -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --help  -h  Show this message and exit.                                                                              -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze self-upgrade [OPTIONS] + +Self upgrade Breeze. + +╭─ Self-upgrade flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ +--use-current-airflow-sources-aUse current workdir Airflow sources for upgrade. +--force-fForce upgrade without asking question to the user. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-setup-autocomplete.svg b/images/breeze/output-setup-autocomplete.svg index b07eca2b16701..de7c17ab43ee5 100644 --- a/images/breeze/output-setup-autocomplete.svg +++ b/images/breeze/output-setup-autocomplete.svg @@ -1,4 +1,4 @@ - + - Command: setup-autocomplete - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: setup-autocomplete + + + + + - + -                                                                                                                          - Usage: breeze setup-autocomplete [OPTIONS]                                                                              -                                                                                                                         - Enables autocompletion of breeze commands.                                                                              -                                                                                                                         -╭─ Setup autocomplete flags ───────────────────────────────────────────────────────────────────────────────────────────╮ -  --force  -f  Force autocomplete setup even if already setup before (overrides the setup).                            -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --answer   -a  Force answer to questions. (y | n | q | yes | no | quit)                                              -  --dry-run  -D  If dry-run is set, commands are only printed, not executed.                                           -  --verbose  -v  Print verbose information about performed steps.                                                      -  --help     -h  Show this message and exit.                                                                           -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze setup-autocomplete [OPTIONS] + +Enables autocompletion of breeze commands. + +╭─ Setup autocomplete flags ───────────────────────────────────────────────────────────────────────────────────────────╮ +--force-fForce autocomplete setup even if already setup before (overrides the setup). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--verbose-vPrint verbose information about performed steps. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-shell.svg b/images/breeze/output-shell.svg index c4a63835bc624..f279eab2d729e 100644 --- a/images/breeze/output-shell.svg +++ b/images/breeze/output-shell.svg @@ -1,4 +1,4 @@ - + - Command: shell - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: shell + + + + + - + -                                                                                                                          - Usage: breeze shell [OPTIONS] [EXTRA_ARGS]...                                                                           -                                                                                                                         - Enter breeze.py environment. this is the default command use when no other is selected.                                 -                                                                                                                         -╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --python               -p  Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10)   -                             [default: 3.7]                                                                            -  --backend              -b  Database backend to use. (>sqlite< | mysql | postgres | mssql) [default: sqlite]          -  --postgres-version     -P  Version of Postgres used. (>10< | 11 | 12 | 13 | 14) [default: 10]                        -  --mysql-version        -M  Version of MySQL used. (>5.7< | 8) [default: 5.7]                                         -  --mssql-version        -S  Version of MsSQL used. (>2017-latest< | 2019-latest) [default: 2017-latest]               -  --integration              Integration(s) to enable when running (can be more than one).                             -                             (cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino |    -                             all)                                                                                      -  --forward-credentials  -f  Forward local credentials to container when running.                                      -  --db-reset             -d  Reset DB when entering the container.                                                     -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Advanced flag for running ──────────────────────────────────────────────────────────────────────────────────────────╮ -  --use-airflow-version       Use (reinstall at entry) Airflow version from PyPI. It can also be `none`, `wheel`, or   -                              `sdist` if Airflow should be removed, installed from wheel packages or sdist packages    -                              available in dist folder respectively. Implies --mount-sources `none`.                   -                              (none | wheel | sdist | <airflow_version>)                                               -  --airflow-extras            Airflow extras to install when --use-airflow-version is used (TEXT)                      -  --use-packages-from-dist    Install all found packages (--package-format determines type) from 'dist' folder when    -                              entering breeze.                                                                         -  --package-format            Format of packages that should be installed from dist. (wheel | sdist)                   -                              [default: wheel]                                                                         -  --force-build               Force image build no matter if it is determined as needed.                               -  --mount-sources             Choose scope of local sources should be mounted (default = selected).                    -                              (selected | all | none)                                                                  -                              [default: selected]                                                                      -  --debian-version            Debian version used for the image. (bullseye | buster) [default: bullseye]               -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose                        -v  Print verbose information about performed steps.                                -  --dry-run                        -D  If dry-run is set, commands are only printed, not executed.                     -  --github-repository              -g  GitHub repository used to pull, push run images. (TEXT)                         -                                       [default: apache/airflow]                                                       -  --airflow-constraints-reference      Constraint reference to use. Useful with --use-airflow-version parameter to     -                                       specify constraints for the installed version and to find newer dependencies    -                                       (TEXT)                                                                          -  --answer                         -a  Force answer to questions. (y | n | q | yes | no | quit)                        -  --help                           -h  Show this message and exit.                                                     -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze shell [OPTIONS] [EXTRA_ARGS]... + +Enter breeze.py environment. this is the default command use when no other is selected. + +╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] +--postgres-version-PVersion of Postgres used.(>10< | 11 | 12 | 13 | 14)[default: 10] +--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] +--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] +--integrationIntegration(s) to enable when running (can be more than one).                             +(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino |    +all)                                                                                      +--forward-credentials-fForward local credentials to container when running. +--db-reset-dReset DB when entering the container. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Advanced flag for running ──────────────────────────────────────────────────────────────────────────────────────────╮ +--use-airflow-versionUse (reinstall at entry) Airflow version from PyPI. It can also be `none`, `wheel`, or   +`sdist` if Airflow should be removed, installed from wheel packages or sdist packages    +available in dist folder respectively. Implies --mount-sources `none`.                   +(none | wheel | sdist | <airflow_version>)                                               +--airflow-extrasAirflow extras to install when --use-airflow-version is used(TEXT) +--use-packages-from-distInstall all found packages (--package-format determines type) from 'dist' folder when    +entering breeze.                                                                         +--package-formatFormat of packages that should be installed from dist.(wheel | sdist)[default: wheel] +--force-buildForce image build no matter if it is determined as needed. +--mount-sourcesChoose scope of local sources should be mounted (default = selected). +(selected | all | none)                                               +[default: selected]                                                   +--debian-versionDebian version used for the image.(bullseye | buster)[default: bullseye] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT) +[default: apache/airflow]                        +--airflow-constraints-referenceConstraint reference to use. Useful with --use-airflow-version parameter to     +specify constraints for the installed version and to find newer dependencies    +(TEXT)                                                                          +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-start-airflow.svg b/images/breeze/output-start-airflow.svg index 6f305e940e2fe..159eec8cb29a0 100644 --- a/images/breeze/output-start-airflow.svg +++ b/images/breeze/output-start-airflow.svg @@ -1,4 +1,4 @@ - + - Command: start-airflow - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: start-airflow + + + + + - + -                                                                                                                          - Usage: breeze start-airflow [OPTIONS] [EXTRA_ARGS]...                                                                   -                                                                                                                         - Enter breeze.py environment and starts all Airflow components in the tmux session.                                      -                                                                                                                         -╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --python                    -p  Python major/minor version used in Airflow image for images.                         -                                  (>3.7< | 3.8 | 3.9 | 3.10)                                                           -                                  [default: 3.7]                                                                       -  --load-example-dags         -e  Enable configuration to load example DAGs when starting Airflow.                     -  --load-default-connections  -c  Enable configuration to load default connections when starting Airflow.              -  --backend                   -b  Database backend to use. (>sqlite< | mysql | postgres | mssql) [default: sqlite]     -  --postgres-version          -P  Version of Postgres used. (>10< | 11 | 12 | 13 | 14) [default: 10]                   -  --mysql-version             -M  Version of MySQL used. (>5.7< | 8) [default: 5.7]                                    -  --mssql-version             -S  Version of MsSQL used. (>2017-latest< | 2019-latest) [default: 2017-latest]          -  --integration                   Integration(s) to enable when running (can be more than one).                        -                                  (cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd |       -                                  trino | all)                                                                         -  --forward-credentials       -f  Forward local credentials to container when running.                                 -  --db-reset                  -d  Reset DB when entering the container.                                                -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Advanced flag for running ──────────────────────────────────────────────────────────────────────────────────────────╮ -  --use-airflow-version       Use (reinstall at entry) Airflow version from PyPI. It can also be `none`, `wheel`, or   -                              `sdist` if Airflow should be removed, installed from wheel packages or sdist packages    -                              available in dist folder respectively. Implies --mount-sources `none`.                   -                              (none | wheel | sdist | <airflow_version>)                                               -  --airflow-extras            Airflow extras to install when --use-airflow-version is used (TEXT)                      -  --use-packages-from-dist    Install all found packages (--package-format determines type) from 'dist' folder when    -                              entering breeze.                                                                         -  --package-format            Format of packages that should be installed from dist. (wheel | sdist)                   -                              [default: wheel]                                                                         -  --force-build               Force image build no matter if it is determined as needed.                               -  --mount-sources             Choose scope of local sources should be mounted (default = selected).                    -                              (selected | all | none)                                                                  -                              [default: selected]                                                                      -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --dry-run                        -D  If dry-run is set, commands are only printed, not executed.                     -  --github-repository              -g  GitHub repository used to pull, push run images. (TEXT)                         -                                       [default: apache/airflow]                                                       -  --airflow-constraints-reference      Constraint reference to use. Useful with --use-airflow-version parameter to     -                                       specify constraints for the installed version and to find newer dependencies    -                                       (TEXT)                                                                          -  --answer                         -a  Force answer to questions. (y | n | q | yes | no | quit)                        -  --verbose                        -v  Print verbose information about performed steps.                                -  --help                           -h  Show this message and exit.                                                     -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze start-airflow [OPTIONS] [EXTRA_ARGS]... + +Enter breeze.py environment and starts all Airflow components in the tmux session. + +╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images. +(>3.7< | 3.8 | 3.9 | 3.10)                                   +[default: 3.7]                                               +--load-example-dags-eEnable configuration to load example DAGs when starting Airflow. +--load-default-connections-cEnable configuration to load default connections when starting Airflow. +--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] +--postgres-version-PVersion of Postgres used.(>10< | 11 | 12 | 13 | 14)[default: 10] +--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] +--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] +--integrationIntegration(s) to enable when running (can be more than one).                        +(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino +| all)                                                                               +--forward-credentials-fForward local credentials to container when running. +--db-reset-dReset DB when entering the container. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Advanced flag for running ──────────────────────────────────────────────────────────────────────────────────────────╮ +--use-airflow-versionUse (reinstall at entry) Airflow version from PyPI. It can also be `none`, `wheel`, or   +`sdist` if Airflow should be removed, installed from wheel packages or sdist packages    +available in dist folder respectively. Implies --mount-sources `none`.                   +(none | wheel | sdist | <airflow_version>)                                               +--airflow-extrasAirflow extras to install when --use-airflow-version is used(TEXT) +--use-packages-from-distInstall all found packages (--package-format determines type) from 'dist' folder when    +entering breeze.                                                                         +--package-formatFormat of packages that should be installed from dist.(wheel | sdist)[default: wheel] +--force-buildForce image build no matter if it is determined as needed. +--mount-sourcesChoose scope of local sources should be mounted (default = selected). +(selected | all | none)                                               +[default: selected]                                                   +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT) +[default: apache/airflow]                        +--airflow-constraints-referenceConstraint reference to use. Useful with --use-airflow-version parameter to     +specify constraints for the installed version and to find newer dependencies    +(TEXT)                                                                          +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--verbose-vPrint verbose information about performed steps. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-static-checks.svg b/images/breeze/output-static-checks.svg index e76f1a3f27ea3..edbc2fee71d47 100644 --- a/images/breeze/output-static-checks.svg +++ b/images/breeze/output-static-checks.svg @@ -1,4 +1,4 @@ - + - Command: static-checks - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: static-checks + + + + + - + -                                                                                                                          - Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]...                                                               -                                                                                                                         - Run static checks.                                                                                                      -                                                                                                                         -╭─ Pre-commit flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --type                  -t  Type(s) of the static checks to run (multiple can be added).                             -                              (all | black | blacken-docs | check-airflow-2-1-compatibility |                          -                              check-airflow-config-yaml-consistent | check-airflow-providers-have-extras |             -                              check-apache-license-rat | check-base-operator-usage |                                   -                              check-boring-cyborg-configuration | check-breeze-top-dependencies-limited |              -                              check-builtin-literals | check-changelog-has-no-duplicates |                             -                              check-daysago-import-from-utils | check-docstring-param-types |                          -                              check-executables-have-shebangs | check-extra-packages-references | check-extras-order   -                              | check-for-inclusive-language | check-hooks-apply |                                     -                              check-incorrect-use-of-LoggingMixin | check-integrations-are-consistent |                -                              check-merge-conflict | check-newsfragments-are-valid |                                   -                              check-no-providers-in-core-examples | check-no-relative-imports |                        -                              check-persist-credentials-disabled-in-github-workflows |                                 -                              check-pre-commit-information-consistent | check-provide-create-sessions-imports |        -                              check-provider-yaml-valid | check-providers-init-file-missing |                          -                              check-providers-subpackages-init-file-exist | check-pydevd-left-in-code |                -                              check-revision-heads-map | check-safe-filter-usage-in-html | check-setup-order |         -                              check-start-date-not-used-in-defaults | check-system-tests-present | check-xml |         -                              codespell | debug-statements | detect-private-key | doctoc | end-of-file-fixer |         -                              fix-encoding-pragma | flynt | forbid-tabs | identity | insert-license | isort |          -                              lint-chart-schema | lint-css | lint-dockerfile | lint-helm-chart | lint-javascript |     -                              lint-json-schema | lint-markdown | lint-openapi | mixed-line-ending |                    -                              pretty-format-json | pydocstyle | python-no-log-warn | pyupgrade | rst-backticks |       -                              run-flake8 | run-mypy | run-shellcheck | static-check-autoflake | trailing-whitespace    -                              | update-breeze-file | update-breeze-readme-config-hash | update-extras |                -                              update-in-the-wild-to-be-sorted | update-inlined-dockerfile-scripts |                    -                              update-local-yml-file | update-migration-references | update-providers-dependencies |    -                              update-setup-cfg-file | update-spelling-wordlist-to-be-sorted |                          -                              update-supported-versions | update-vendored-in-k8s-json-schema | update-version |        -                              yamllint | yesqa)                                                                        -  --file                  -f  List of files to run the checks on. (PATH)                                               -  --all-files             -a  Run checks on all files.                                                                 -  --show-diff-on-failure  -s  Show diff for files modified by the checks.                                              -  --last-commit           -c  Run checks for all files in last commit. Mutually exclusive with --commit-ref.           -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --commit-ref         -r  Run checks for this commit reference only (can be any git commit-ish reference). Mutually   -                           exclusive with --last-commit.                                                               -                           (TEXT)                                                                                      -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]... + +Run static checks. + +╭─ Pre-commit flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--type-tType(s) of the static checks to run (multiple can be added).                             +(all | black | blacken-docs | check-airflow-2-1-compatibility |                          +check-airflow-config-yaml-consistent | check-airflow-providers-have-extras |             +check-apache-license-rat | check-base-operator-usage | check-boring-cyborg-configuration +| check-breeze-top-dependencies-limited | check-builtin-literals |                       +check-changelog-has-no-duplicates | check-daysago-import-from-utils |                    +check-docstring-param-types | check-executables-have-shebangs |                          +check-extra-packages-references | check-extras-order | check-for-inclusive-language |    +check-hooks-apply | check-incorrect-use-of-LoggingMixin |                                +check-integrations-are-consistent | check-merge-conflict | check-newsfragments-are-valid +| check-no-providers-in-core-examples | check-no-relative-imports |                      +check-persist-credentials-disabled-in-github-workflows |                                 +check-pre-commit-information-consistent | check-provide-create-sessions-imports |        +check-provider-yaml-valid | check-providers-init-file-missing |                          +check-providers-subpackages-init-file-exist | check-pydevd-left-in-code |                +check-revision-heads-map | check-safe-filter-usage-in-html | check-setup-order |         +check-start-date-not-used-in-defaults | check-system-tests-present | check-xml |         +codespell | debug-statements | detect-private-key | doctoc | end-of-file-fixer |         +fix-encoding-pragma | flynt | forbid-tabs | identity | insert-license | isort |          +lint-chart-schema | lint-css | lint-dockerfile | lint-helm-chart | lint-javascript |     +lint-json-schema | lint-markdown | lint-openapi | mixed-line-ending | pretty-format-json +| pydocstyle | python-no-log-warn | pyupgrade | rst-backticks | run-flake8 | run-mypy |  +run-shellcheck | static-check-autoflake | trailing-whitespace | update-breeze-file |     +update-breeze-readme-config-hash | update-extras | update-in-the-wild-to-be-sorted |     +update-inlined-dockerfile-scripts | update-local-yml-file | update-migration-references  +| update-providers-dependencies | update-setup-cfg-file |                                +update-spelling-wordlist-to-be-sorted | update-supported-versions |                      +update-vendored-in-k8s-json-schema | update-version | yamllint | yesqa)                  +--file-fList of files to run the checks on.(PATH) +--all-files-aRun checks on all files. +--show-diff-on-failure-sShow diff for files modified by the checks. +--last-commit-cRun checks for all files in last commit. Mutually exclusive with --commit-ref. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--commit-ref-rRun checks for this commit reference only (can be any git commit-ish reference). Mutually   +exclusive with --last-commit.                                                               +(TEXT)                                                                                      +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-stop.svg b/images/breeze/output-stop.svg index 6f5a9ca2d86d7..5fc1819e95293 100644 --- a/images/breeze/output-stop.svg +++ b/images/breeze/output-stop.svg @@ -1,4 +1,4 @@ - + - Command: stop - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: stop + + + + + - + -                                                                                                                          - Usage: breeze stop [OPTIONS]                                                                                            -                                                                                                                         - Stop running breeze environment.                                                                                        -                                                                                                                         -╭─ Stop flags ─────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --preserve-volumes  -p  Skip removing volumes when stopping Breeze.                                                  -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose  -v  Print verbose information about performed steps.                                                      -  --dry-run  -D  If dry-run is set, commands are only printed, not executed.                                           -  --help     -h  Show this message and exit.                                                                           -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze stop [OPTIONS] + +Stop running breeze environment. + +╭─ Stop flags ─────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--preserve-volumes-pSkip removing volumes when stopping Breeze. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-tests.svg b/images/breeze/output-tests.svg index 0b344da888747..7c02458342214 100644 --- a/images/breeze/output-tests.svg +++ b/images/breeze/output-tests.svg @@ -1,4 +1,4 @@ - + - Command: tests - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: tests + + + + + - + -                                                                                                                          - Usage: breeze tests [OPTIONS] [EXTRA_PYTEST_ARGS]...                                                                    -                                                                                                                         - Run the specified unit test targets. Multiple targets may be specified separated by spaces.                             -                                                                                                                         -╭─ Basic flag for tests command ───────────────────────────────────────────────────────────────────────────────────────╮ -  --integration       Integration(s) to enable when running (can be more than one).                                    -                      (cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino | all)      -  --test-type    -tt  Type of test to run.                                                                             -                      (All | Always | Core | Providers | API | CLI | Integration | Other | WWW | Postgres | MySQL |    -                      Helm | Quarantined)                                                                              -  --db-reset     -d   Reset DB when entering the container.                                                            -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --dry-run  -D  If dry-run is set, commands are only printed, not executed.                                           -  --verbose  -v  Print verbose information about performed steps.                                                      -  --help     -h  Show this message and exit.                                                                           -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze tests [OPTIONS] [EXTRA_PYTEST_ARGS]... + +Run the specified unit test targets. Multiple targets may be specified separated by spaces. + +╭─ Basic flag for tests command ───────────────────────────────────────────────────────────────────────────────────────╮ +--integrationIntegration(s) to enable when running (can be more than one).                               +(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino | all) +--test-type-ttType of test to run.                                                                             +(All | Always | Core | Providers | API | CLI | Integration | Other | WWW | Postgres | MySQL |    +Helm | Quarantined)                                                                              +--db-reset-dReset DB when entering the container. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--verbose-vPrint verbose information about performed steps. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-verify-image.svg b/images/breeze/output-verify-image.svg index e35e11819f8ed..b79efb93f2602 100644 --- a/images/breeze/output-verify-image.svg +++ b/images/breeze/output-verify-image.svg @@ -1,4 +1,4 @@ - + - Command: verify-image - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: verify-image + + + + + - + -                                                                                                                          - Usage: breeze verify-image [OPTIONS] [EXTRA_PYTEST_ARGS]...                                                             -                                                                                                                         - Verify CI image.                                                                                                        -                                                                                                                         -╭─ Verify image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --image-name  -n  Name of the image to verify (overrides --python and --image-tag). (TEXT)                           -  --python      -p  Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10)            -                    [default: 3.7]                                                                                     -  --image-tag   -t  Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT)                      -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze verify-image [OPTIONS] [EXTRA_PYTEST_ARGS]... + +Verify CI image. + +╭─ Verify image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ +--image-name-nName of the image to verify (overrides --python and --image-tag).(TEXT) +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--image-tag-tTag added to the default naming conventions of Airflow CI/PROD images.(TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-verify-prod-image.svg b/images/breeze/output-verify-prod-image.svg index 3dab7852ead6a..4bc7a36527741 100644 --- a/images/breeze/output-verify-prod-image.svg +++ b/images/breeze/output-verify-prod-image.svg @@ -1,4 +1,4 @@ - + - Command: verify-prod-image - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: verify-prod-image + + + + + - + -                                                                                                                          - Usage: breeze verify-prod-image [OPTIONS] [EXTRA_PYTEST_ARGS]...                                                        -                                                                                                                         - Verify Production image.                                                                                                -                                                                                                                         -╭─ Verify image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --image-name  -n  Name of the image to verify (overrides --python and --image-tag). (TEXT)                           -  --python      -p  Python major/minor version used in Airflow image for images. (>3.7< | 3.8 | 3.9 | 3.10)            -                    [default: 3.7]                                                                                     -  --image-tag   -t  Tag added to the default naming conventions of Airflow CI/PROD images. (TEXT)                      -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze verify-prod-image [OPTIONS] [EXTRA_PYTEST_ARGS]... + +Verify Production image. + +╭─ Verify image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ +--image-name-nName of the image to verify (overrides --python and --image-tag).(TEXT) +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--image-tag-tTag added to the default naming conventions of Airflow CI/PROD images.(TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-verify-provider-packages.svg b/images/breeze/output-verify-provider-packages.svg index 061622d35349f..cfb88fbcc4755 100644 --- a/images/breeze/output-verify-provider-packages.svg +++ b/images/breeze/output-verify-provider-packages.svg @@ -1,4 +1,4 @@ - + - Command: verify-provider-packages - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: verify-provider-packages + + + + + - + -                                                                                                                          - Usage: breeze verify-provider-packages [OPTIONS]                                                                        -                                                                                                                         - Verifies if all provider code is following expectations for providers.                                                  -                                                                                                                         -╭─ Provider verification flags ────────────────────────────────────────────────────────────────────────────────────────╮ -  --use-airflow-version              Use (reinstall at entry) Airflow version from PyPI. It can also be `none`,        -                                     `wheel`, or `sdist` if Airflow should be removed, installed from wheel packages   -                                     or sdist packages available in dist folder respectively. Implies                  -                                     --mount-sources `none`.                                                           -                                     (none | wheel | sdist | <airflow_version>)                                        -  --airflow-constraints-reference    Constraint reference to use. Useful with --use-airflow-version parameter to       -                                     specify constraints for the installed version and to find newer dependencies      -                                     (TEXT)                                                                            -  --airflow-extras                   Airflow extras to install when --use-airflow-version is used (TEXT)               -  --use-packages-from-dist           Install all found packages (--package-format determines type) from 'dist'         -                                     folder when entering breeze.                                                      -  --package-format                   Format of packages that should be installed from dist. (wheel | sdist)            -                                     [default: wheel]                                                                  -  --debug                            Drop user in shell instead of running the command. Useful for debugging.          -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze verify-provider-packages [OPTIONS] + +Verifies if all provider code is following expectations for providers. + +╭─ Provider verification flags ────────────────────────────────────────────────────────────────────────────────────────╮ +--use-airflow-versionUse (reinstall at entry) Airflow version from PyPI. It can also be `none`,        +`wheel`, or `sdist` if Airflow should be removed, installed from wheel packages   +or sdist packages available in dist folder respectively. Implies --mount-sources +`none`.                                                                           +(none | wheel | sdist | <airflow_version>)                                        +--airflow-constraints-referenceConstraint reference to use. Useful with --use-airflow-version parameter to       +specify constraints for the installed version and to find newer dependencies      +(TEXT)                                                                            +--airflow-extrasAirflow extras to install when --use-airflow-version is used(TEXT) +--use-packages-from-distInstall all found packages (--package-format determines type) from 'dist' folder  +when entering breeze.                                                             +--package-formatFormat of packages that should be installed from dist.(wheel | sdist) +[default: wheel]                                       +--debugDrop user in shell instead of running the command. Useful for debugging. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + diff --git a/images/breeze/output-version.svg b/images/breeze/output-version.svg index d136f095946c2..9d37a0a7ac852 100644 --- a/images/breeze/output-version.svg +++ b/images/breeze/output-version.svg @@ -1,4 +1,4 @@ - + - Command: version - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: version + + + + + - + -                                                                                                                          - Usage: breeze version [OPTIONS]                                                                                         -                                                                                                                         - Print information about version of apache-airflow-breeze.                                                               -                                                                                                                         -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --verbose  -v  Print verbose information about performed steps.                                                      -  --help     -h  Show this message and exit.                                                                           -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - + + +Usage: breeze version [OPTIONS] + +Print information about version of apache-airflow-breeze. + +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + From 2a621197cccbcd3f233bddc7abe2e18c07bc6c9f Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 31 May 2022 22:53:02 +0200 Subject: [PATCH 04/13] Oops --- tests/providers/tabular/hooks/test_tabular.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/providers/tabular/hooks/test_tabular.py b/tests/providers/tabular/hooks/test_tabular.py index b3c68452e5cfa..639c43cded746 100644 --- a/tests/providers/tabular/hooks/test_tabular.py +++ b/tests/providers/tabular/hooks/test_tabular.py @@ -28,7 +28,7 @@ class TestTabularHook(unittest.TestCase): def test_tabular(self): access_token = 'eyJ0eXAiOiJKV1QiLCJhbGciOiJSU' with requests_mock.Mocker() as m: - m.get( + m.post( 'https://api.tabulardata.io/ws/v1/oauth/tokens', json={ 'access_token': access_token, From ce560938e6aafc8a7422c0d13c704f63d836eb21 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 14 Jun 2022 09:04:24 +0200 Subject: [PATCH 05/13] Rebuilding the docs --- airflow/utils/db.py | 2 +- .../connections.rst | 6 +- images/breeze/output-build-docs.svg | 284 +++++++++++++ images/breeze/output-build-image.svg | 292 +++++++++++++ images/breeze/output-commands-hash.txt | 2 + images/breeze/output-commands.svg | 396 ++++++++++++++++++ .../output-prepare-provider-documentation.svg | 168 ++++++++ .../output-prepare-provider-packages.svg | 141 ------- images/breeze/output-static-checks.svg | 260 ++++++++++++ 9 files changed, 1405 insertions(+), 146 deletions(-) create mode 100644 images/breeze/output-build-docs.svg create mode 100644 images/breeze/output-build-image.svg create mode 100644 images/breeze/output-commands-hash.txt create mode 100644 images/breeze/output-commands.svg create mode 100644 images/breeze/output-prepare-provider-documentation.svg create mode 100644 images/breeze/output-static-checks.svg diff --git a/airflow/utils/db.py b/airflow/utils/db.py index e7a632756c869..6523643911b25 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -610,7 +610,7 @@ def create_default_connections(session: Session = NEW_SESSION): Connection( conn_id="tabular_default", conn_type="tabular", - extra='{"extra__tabular__baseUrl": "https://api.tabulardata.io/ws/v1"}', + host="https://api.tabulardata.io/ws/v1", ), session, ) diff --git a/docs/apache-airflow-providers-tabular/connections.rst b/docs/apache-airflow-providers-tabular/connections.rst index 2bbbe45e4116b..ff73cd726778c 100644 --- a/docs/apache-airflow-providers-tabular/connections.rst +++ b/docs/apache-airflow-providers-tabular/connections.rst @@ -35,7 +35,5 @@ Client ID Client Secret The Client Secret from Tabular -Extra (optional, connection parameters) - Specify the extra parameters (as json dictionary) that can be used the Tabular connection. The following parameters out of the standard python parameters are supported: - - * ``base_url`` - Optional, sets the URL to the Tabular environment. By default `https://api.tabulardata.io/ws/v1` +Host + Sets the URL to the Tabular environment. By default `https://api.tabulardata.io/ws/v1` diff --git a/images/breeze/output-build-docs.svg b/images/breeze/output-build-docs.svg new file mode 100644 index 0000000000000..ab86dd44a54b7 --- /dev/null +++ b/images/breeze/output-build-docs.svg @@ -0,0 +1,284 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: build-docs + + + + + + + + + + +Usage: breeze build-docs [OPTIONS] + +Build documentation in the container. + +╭─ Doc flags ──────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--docs-only-dOnly build documentation. +--spellcheck-only-sOnly run spell checking. +--for-production-pBuilds documentation for official release i.e. all links point to stable version. +--package-filter-pList of packages to consider.                                                                 +(apache-airflow | apache-airflow-providers | apache-airflow-providers-airbyte |               +apache-airflow-providers-alibaba | apache-airflow-providers-amazon |                          +apache-airflow-providers-apache-beam | apache-airflow-providers-apache-cassandra |            +apache-airflow-providers-apache-drill | apache-airflow-providers-apache-druid |               +apache-airflow-providers-apache-hdfs | apache-airflow-providers-apache-hive |                 +apache-airflow-providers-apache-kylin | apache-airflow-providers-apache-livy |                +apache-airflow-providers-apache-pig | apache-airflow-providers-apache-pinot |                 +apache-airflow-providers-apache-spark | apache-airflow-providers-apache-sqoop |               +apache-airflow-providers-arangodb | apache-airflow-providers-asana |                          +apache-airflow-providers-celery | apache-airflow-providers-cloudant |                         +apache-airflow-providers-cncf-kubernetes | apache-airflow-providers-databricks |              +apache-airflow-providers-datadog | apache-airflow-providers-dbt-cloud |                       +apache-airflow-providers-dingding | apache-airflow-providers-discord |                        +apache-airflow-providers-docker | apache-airflow-providers-elasticsearch |                    +apache-airflow-providers-exasol | apache-airflow-providers-facebook |                         +apache-airflow-providers-ftp | apache-airflow-providers-github |                              +apache-airflow-providers-google | apache-airflow-providers-grpc |                             +apache-airflow-providers-hashicorp | apache-airflow-providers-http |                          +apache-airflow-providers-imap | apache-airflow-providers-influxdb |                           +apache-airflow-providers-jdbc | apache-airflow-providers-jenkins |                            +apache-airflow-providers-jira | apache-airflow-providers-microsoft-azure |                    +apache-airflow-providers-microsoft-mssql | apache-airflow-providers-microsoft-psrp |          +apache-airflow-providers-microsoft-winrm | apache-airflow-providers-mongo |                   +apache-airflow-providers-mysql | apache-airflow-providers-neo4j |                             +apache-airflow-providers-odbc | apache-airflow-providers-openfaas |                           +apache-airflow-providers-opsgenie | apache-airflow-providers-oracle |                         +apache-airflow-providers-pagerduty | apache-airflow-providers-papermill |                     +apache-airflow-providers-plexus | apache-airflow-providers-postgres |                         +apache-airflow-providers-presto | apache-airflow-providers-qubole |                           +apache-airflow-providers-redis | apache-airflow-providers-salesforce |                        +apache-airflow-providers-samba | apache-airflow-providers-segment |                           +apache-airflow-providers-sendgrid | apache-airflow-providers-sftp |                           +apache-airflow-providers-singularity | apache-airflow-providers-slack |                       +apache-airflow-providers-snowflake | apache-airflow-providers-sqlite |                        +apache-airflow-providers-ssh | apache-airflow-providers-tableau |                             +apache-airflow-providers-tabular | apache-airflow-providers-telegram |                        +apache-airflow-providers-trino | apache-airflow-providers-vertica |                           +apache-airflow-providers-yandex | apache-airflow-providers-zendesk | docker-stack |           +helm-chart)                                                                                   +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + + + diff --git a/images/breeze/output-build-image.svg b/images/breeze/output-build-image.svg new file mode 100644 index 0000000000000..34c28afd8b91b --- /dev/null +++ b/images/breeze/output-build-image.svg @@ -0,0 +1,292 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: build-image + + + + + + + + + + +Usage: breeze build-image [OPTIONS] + +Build CI image. Include building multiple images for all python versions (sequentially). + +╭─ Basic usage ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images. +(>3.7< | 3.8 | 3.9 | 3.10)                                   +[default: 3.7]                                               +--upgrade-to-newer-dependencies-uWhen set, upgrade all PIP packages to latest. +--debian-versionDebian version used for the image.(bullseye | buster)[default: bullseye] +--image-tag-tTag added to the default naming conventions of Airflow CI/PROD images.(TEXT) +--tag-as-latestTags the image as latest and update checksum of all files after pulling. Useful +when you build or pull image with --image-tag.                                  +--docker-cache-cCache option for image used during the build.(registry | local | disabled) +[default: registry]                           +--force-buildForce image build no matter if it is determined as needed. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Building multiple images ───────────────────────────────────────────────────────────────────────────────────────────╮ +--build-multiple-imagesRun the operation sequentially on all or selected subset of Python versions. +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Advanced options (for power users) ─────────────────────────────────────────────────────────────────────────────────╮ +--install-providers-from-sourcesInstall providers from sources when installing. +--airflow-constraints-modeMode of constraints for CI image building                               +(constraints-source-providers | constraints | constraints-no-providers) +[default: constraints-source-providers]                                 +--airflow-constraints-referenceConstraint reference to use when building the image.(TEXT) +--additional-python-depsAdditional python dependencies to use when building the images.(TEXT) +--runtime-apt-depsApt runtime dependencies to use when building the images.(TEXT) +--runtime-apt-commandCommand executed before runtime apt deps are installed.(TEXT) +--additional-extrasAdditional extra package while installing Airflow in the image.(TEXT) +--additional-runtime-apt-depsAdditional apt runtime dependencies to use when building the images.(TEXT) +--additional-runtime-apt-envAdditional environment variables set when adding runtime dependencies.(TEXT) +--additional-runtime-apt-commandAdditional command executed before runtime apt deps are installed.(TEXT) +--additional-dev-apt-depsAdditional apt dev dependencies to use when building the images.(TEXT) +--additional-dev-apt-envAdditional environment variables set when adding dev dependencies.(TEXT) +--additional-dev-apt-commandAdditional command executed before dev apt deps are installed.(TEXT) +--dev-apt-depsApt dev dependencies to use when building the images.(TEXT) +--dev-apt-commandCommand executed before dev apt deps are installed.(TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Preparing cache and push (for maintainers and CI) ──────────────────────────────────────────────────────────────────╮ +--github-tokenThe token used to authenticate to GitHub.(TEXT) +--github-usernameThe user name used to authenticate to GitHub.(TEXT) +--platformPlatform for Airflow image.(linux/amd64 | linux/arm64 | linux/amd64,linux/arm64) +--push-imagePush image after building it. +--empty-imagePrepare empty image tagged with the same name as the Airflow image. +--prepare-buildx-cachePrepares build cache (this is done as separate per-platform steps instead of building the  +image).                                                                                    +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + + + diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt new file mode 100644 index 0000000000000..f1772f3464e1b --- /dev/null +++ b/images/breeze/output-commands-hash.txt @@ -0,0 +1,2 @@ +mypy-cache-volume +a862d3c76df14e00a5bf68ab899f8be8 diff --git a/images/breeze/output-commands.svg b/images/breeze/output-commands.svg new file mode 100644 index 0000000000000..a592f4642e045 --- /dev/null +++ b/images/breeze/output-commands.svg @@ -0,0 +1,396 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Breeze commands + + + + + + + + + + +Usage: breeze [OPTIONS] COMMAND [ARGS]... + +╭─ Basic flags for the default (shell) command ────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] +--postgres-version-PVersion of Postgres used.(>10< | 11 | 12 | 13 | 14)[default: 10] +--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] +--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] +--integrationIntegration(s) to enable when running (can be more than one).                             +(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino |    +all)                                                                                      +--forward-credentials-fForward local credentials to container when running. +--db-reset-dReset DB when entering the container. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Advanced flags for the default (shell) command ─────────────────────────────────────────────────────────────────────╮ +--use-airflow-versionUse (reinstall at entry) Airflow version from PyPI. It can also be `none`, `wheel`, or   +`sdist` if Airflow should be removed, installed from wheel packages or sdist packages    +available in dist folder respectively. Implies --mount-sources `none`.                   +(none | wheel | sdist | <airflow_version>)                                               +--airflow-extrasAirflow extras to install when --use-airflow-version is used(TEXT) +--use-packages-from-distInstall all found packages (--package-format determines type) from 'dist' folder when    +entering breeze.                                                                         +--package-formatFormat of packages that should be installed from dist.(wheel | sdist)[default: wheel] +--force-buildForce image build no matter if it is determined as needed. +--mount-sourcesChoose scope of local sources should be mounted (default = selected). +(selected | all | none)                                               +[default: selected]                                                   +--debian-versionDebian version used for the image.(bullseye | buster)[default: bullseye] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Developer tools ────────────────────────────────────────────────────────────────────────────────────────────────────╮ +shell          Enter breeze.py environment. this is the default command use when no other is selected.             +start-airflow  Enter breeze.py environment and starts all Airflow components in the tmux session.                  +exec           Joins the interactive shell of running airflow container                                            +stop           Stop running breeze environment.                                                                    +build-docs     Build documentation in the container.                                                               +static-checks  Run static checks.                                                                                  +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Testing ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +docker-compose-tests Run docker-compose tests.                                                                     +tests                Run the specified unit test targets. Multiple targets may be specified separated by spaces.   +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Configuration & maintenance ────────────────────────────────────────────────────────────────────────────────────────╮ +cleanup                  Cleans the cache of parameters, docker cache and optionally - currently downloaded        +images.                                                                                   +self-upgrade             Self upgrade Breeze.                                                                      +setup-autocomplete       Enables autocompletion of breeze commands.                                                +config                   Show/update configuration (Python, Backend, Cheatsheet, ASCIIART).                        +resource-check           Check if available docker resources are enough.                                           +free-space               Free space for jobs run in CI.                                                            +fix-ownership            Fix ownership of source files to be same as host user.                                    +regenerate-command-imagesRegenerate breeze command images.                                                         +command-hash-export      Outputs hash of all click commands to file or stdout if `-` is used (useful to see if     +images should be regenerated).                                                            +version                  Print information about version of apache-airflow-breeze.                                 +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ CI Image tools ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +build-image   Build CI image. Include building multiple images for all python versions (sequentially).             +pull-image    Pull and optionally verify CI images - possibly in parallel for all Python versions.                 +verify-image  Verify CI image.                                                                                     +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Production Image tools ─────────────────────────────────────────────────────────────────────────────────────────────╮ +build-prod-image Build Production image. Include building multiple images for all or selected Python versions      +sequentially.                                                                                     +pull-prod-image  Pull and optionally verify Production images - possibly in parallel for all Python versions.      +verify-prod-imageVerify Production image.                                                                          +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Release management ─────────────────────────────────────────────────────────────────────────────────────────────────╮ +verify-provider-packages         Verifies if all provider code is following expectations for providers.            +prepare-provider-documentation   Prepare CHANGELOG, README and COMMITS information for providers.                  +prepare-provider-packages        Prepare sdist/whl packages of Airflow Providers.                                  +prepare-airflow-package          Prepare sdist/whl package of Airflow.                                             +release-prod-images              Release production images to DockerHub (needs DockerHub permissions).             +generate-constraints             Generates pinned constraint files with all extras from setup.py in parallel.      +find-newer-dependencies          Finds which dependencies are being upgraded.                                      +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + + + diff --git a/images/breeze/output-prepare-provider-documentation.svg b/images/breeze/output-prepare-provider-documentation.svg new file mode 100644 index 0000000000000..f0f660bf5d42b --- /dev/null +++ b/images/breeze/output-prepare-provider-documentation.svg @@ -0,0 +1,168 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: prepare-provider-documentation + + + + + + + + + + +Usage: breeze prepare-provider-documentation [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra | +                                             apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin | +                                             apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop | +                                             arangodb | asana | celery | cloudant | cncf.kubernetes | databricks | +                                             datadog | dbt.cloud | dingding | discord | docker | elasticsearch | +                                             exasol | facebook | ftp | github | google | grpc | hashicorp | http | +                                             imap | influxdb | jdbc | jenkins | jira | microsoft.azure | +                                             microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql | +                                             neo4j | odbc | openfaas | opsgenie | oracle | pagerduty | papermill | +                                             plexus | postgres | presto | qubole | redis | salesforce | samba | +                                             segment | sendgrid | sftp | singularity | slack | snowflake | sqlite | +                                             ssh | tableau | tabular | telegram | trino | vertica | yandex | +                                             zendesk]... + +Prepare CHANGELOG, README and COMMITS information for providers. + +╭─ Provider documentation preparation flags ───────────────────────────────────────────────────────────────────────────╮ +--debugDrop user in shell instead of running the command. Useful for debugging. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + + + diff --git a/images/breeze/output-prepare-provider-packages.svg b/images/breeze/output-prepare-provider-packages.svg index 43648cfc20c85..091257db2bd9a 100644 --- a/images/breeze/output-prepare-provider-packages.svg +++ b/images/breeze/output-prepare-provider-packages.svg @@ -19,28 +19,19 @@ font-weight: 700; } -<<<<<<< HEAD .terminal-3201271100-matrix { -======= - .terminal-3636495061-matrix { ->>>>>>> fe2ef0fcc75fa7be9e1ec1113af260a3a3c9e7ae font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } -<<<<<<< HEAD .terminal-3201271100-title { -======= - .terminal-3636495061-title { ->>>>>>> fe2ef0fcc75fa7be9e1ec1113af260a3a3c9e7ae font-size: 18px; font-weight: bold; font-family: arial; } -<<<<<<< HEAD .terminal-3201271100-r1 { fill: #c5c8c6;font-weight: bold } .terminal-3201271100-r2 { fill: #c5c8c6 } .terminal-3201271100-r3 { fill: #d0b344;font-weight: bold } @@ -133,116 +124,17 @@ -======= - .terminal-3636495061-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-3636495061-r2 { fill: #c5c8c6 } -.terminal-3636495061-r3 { fill: #d0b344;font-weight: bold } -.terminal-3636495061-r4 { fill: #868887 } -.terminal-3636495061-r5 { fill: #68a0b3;font-weight: bold } -.terminal-3636495061-r6 { fill: #8d7b39 } -.terminal-3636495061-r7 { fill: #98a84b;font-weight: bold } - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - ->>>>>>> fe2ef0fcc75fa7be9e1ec1113af260a3a3c9e7ae -<<<<<<< HEAD Command: prepare-provider-packages -======= - Command: prepare-provider-packages ->>>>>>> fe2ef0fcc75fa7be9e1ec1113af260a3a3c9e7ae -<<<<<<< HEAD @@ -274,39 +166,6 @@ --github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] --help-hShow this message and exit. ╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -======= - - - - -Usage: breeze prepare-provider-packages [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra | -                                        apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin | -                                        apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop | -                                        arangodb | asana | celery | cloudant | cncf.kubernetes | databricks | datadog -                                        | dbt.cloud | dingding | discord | docker | elasticsearch | exasol | facebook -                                        | ftp | github | google | grpc | hashicorp | http | imap | influxdb | jdbc | -                                        jenkins | jira | microsoft.azure | microsoft.mssql | microsoft.psrp | -                                        microsoft.winrm | mongo | mysql | neo4j | odbc | openfaas | opsgenie | oracle -                                        | pagerduty | papermill | plexus | postgres | presto | qubole | redis | -                                        salesforce | samba | segment | sendgrid | sftp | singularity | slack | -                                        snowflake | sqlite | ssh | tableau | telegram | trino | vertica | yandex | -                                        zendesk]... - -Prepare sdist/whl packages of Airflow Providers. - -╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---package-formatFormat of packages.(wheel | sdist | both)[default: wheel] ---version-suffix-for-pypiVersion suffix used for PyPI packages (alpha, beta, rc1, etc.).(TEXT) ---package-list-fileRead list of packages from text file (one package per line)(FILENAME) ---debugDrop user in shell instead of running the command. Useful for debugging. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ ->>>>>>> fe2ef0fcc75fa7be9e1ec1113af260a3a3c9e7ae diff --git a/images/breeze/output-static-checks.svg b/images/breeze/output-static-checks.svg new file mode 100644 index 0000000000000..695266c06fd3b --- /dev/null +++ b/images/breeze/output-static-checks.svg @@ -0,0 +1,260 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: static-checks + + + + + + + + + + +Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]... + +Run static checks. + +╭─ Pre-commit flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--type-tType(s) of the static checks to run (multiple can be added).                             +(all | black | blacken-docs | check-airflow-2-1-compatibility |                          +check-airflow-config-yaml-consistent | check-airflow-providers-have-extras |             +check-apache-license-rat | check-base-operator-partial-arguments |                       +check-base-operator-usage | check-boring-cyborg-configuration |                          +check-breeze-top-dependencies-limited | check-builtin-literals |                         +check-changelog-has-no-duplicates | check-daysago-import-from-utils |                    +check-docstring-param-types | check-example-dags-urls | check-executables-have-shebangs  +| check-extra-packages-references | check-extras-order | check-for-inclusive-language |  +check-hooks-apply | check-incorrect-use-of-LoggingMixin |                                +check-integrations-are-consistent | check-merge-conflict | check-newsfragments-are-valid +| check-no-providers-in-core-examples | check-no-relative-imports |                      +check-persist-credentials-disabled-in-github-workflows |                                 +check-pre-commit-information-consistent | check-provide-create-sessions-imports |        +check-provider-yaml-valid | check-providers-init-file-missing |                          +check-providers-subpackages-init-file-exist | check-pydevd-left-in-code |                +check-revision-heads-map | check-safe-filter-usage-in-html | check-setup-order |         +check-start-date-not-used-in-defaults | check-system-tests-present |                     +check-system-tests-tocs | check-xml | codespell | create-missing-init-py-files-tests |   +debug-statements | detect-private-key | doctoc | end-of-file-fixer | fix-encoding-pragma +| flynt | forbid-tabs | identity | insert-license | isort | lint-chart-schema | lint-css +| lint-dockerfile | lint-helm-chart | lint-javascript | lint-json-schema | lint-markdown +| lint-openapi | mixed-line-ending | pretty-format-json | pydocstyle |                   +python-no-log-warn | pyupgrade | rst-backticks | run-flake8 | run-mypy | run-shellcheck  +| static-check-autoflake | trailing-whitespace | update-breeze-file |                    +update-breeze-readme-config-hash | update-extras | update-in-the-wild-to-be-sorted |     +update-inlined-dockerfile-scripts | update-local-yml-file | update-migration-references  +| update-providers-dependencies | update-setup-cfg-file |                                +update-spelling-wordlist-to-be-sorted | update-supported-versions |                      +update-vendored-in-k8s-json-schema | update-version | yamllint | yesqa)                  +--file-fList of files to run the checks on.(PATH) +--all-files-aRun checks on all files. +--show-diff-on-failure-sShow diff for files modified by the checks. +--last-commit-cRun checks for all files in last commit. Mutually exclusive with --commit-ref. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--commit-ref-rRun checks for this commit reference only (can be any git commit-ish reference). Mutually   +exclusive with --last-commit.                                                               +(TEXT)                                                                                      +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + + + From 55346f73d3d3d153451004a826b26ff9d2051a8b Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 14 Jun 2022 16:36:48 +0200 Subject: [PATCH 06/13] Cleanup command hash file --- images/breeze/output-commands-hash.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt index f1772f3464e1b..fe55cd45fb7f7 100644 --- a/images/breeze/output-commands-hash.txt +++ b/images/breeze/output-commands-hash.txt @@ -1,2 +1 @@ -mypy-cache-volume a862d3c76df14e00a5bf68ab899f8be8 From c3f515c0a93ec5dbb268f17a19560e6382a7ee5d Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 16 Jun 2022 18:20:57 +0200 Subject: [PATCH 07/13] Regen the hash file --- images/breeze/output-commands-hash.txt | 1 + 1 file changed, 1 insertion(+) create mode 100644 images/breeze/output-commands-hash.txt diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt new file mode 100644 index 0000000000000..e6b8e1b642379 --- /dev/null +++ b/images/breeze/output-commands-hash.txt @@ -0,0 +1 @@ +723c253be43b6d97139979fd3d627aff From 2bc51e7319a60824f0854f31ab34ac2e47ee188f Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 29 Jun 2022 22:20:32 +0200 Subject: [PATCH 08/13] Update providers --- airflow/providers/tabular/provider.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/tabular/provider.yaml b/airflow/providers/tabular/provider.yaml index ae37da70a15d3..6465b8e338d9e 100644 --- a/airflow/providers/tabular/provider.yaml +++ b/airflow/providers/tabular/provider.yaml @@ -24,7 +24,7 @@ description: | versions: - 0.0.1 -additional-dependencies: +dependencies: - apache-airflow>=2.2.0 integrations: From 5111a5d44ca6bc1c283d57e1e213734bc9a8d2a3 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 29 Jun 2022 23:13:40 +0200 Subject: [PATCH 09/13] Move to the new setup --- generated/provider_dependencies.json | 6 + images/breeze/output-build-docs.svg | 284 ++++++++++++++++++ images/breeze/output-commands-hash.txt | 37 +++ .../output-prepare-provider-documentation.svg | 168 +++++++++++ .../output-prepare-provider-packages.svg | 172 +++++++++++ 5 files changed, 667 insertions(+) create mode 100644 images/breeze/output-build-docs.svg create mode 100644 images/breeze/output-commands-hash.txt create mode 100644 images/breeze/output-prepare-provider-documentation.svg create mode 100644 images/breeze/output-prepare-provider-packages.svg diff --git a/generated/provider_dependencies.json b/generated/provider_dependencies.json index 6226703df35e5..c968d46e9c78b 100644 --- a/generated/provider_dependencies.json +++ b/generated/provider_dependencies.json @@ -634,6 +634,12 @@ ], "cross-providers-deps": [] }, + "tabular": { + "deps": [ + "apache-airflow>=2.2.0" + ], + "cross-providers-deps": [] + }, "telegram": { "deps": [ "apache-airflow>=2.2.0", diff --git a/images/breeze/output-build-docs.svg b/images/breeze/output-build-docs.svg new file mode 100644 index 0000000000000..50c566d055002 --- /dev/null +++ b/images/breeze/output-build-docs.svg @@ -0,0 +1,284 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: build-docs + + + + + + + + + + +Usage: breeze build-docs [OPTIONS] + +Build documentation in the container. + +╭─ Doc flags ──────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--docs-only-dOnly build documentation. +--spellcheck-only-sOnly run spell checking. +--for-production-pBuilds documentation for official release i.e. all links point to stable version. +--package-filter-pList of packages to consider.                                                                 +(apache-airflow | apache-airflow-providers | apache-airflow-providers-airbyte |               +apache-airflow-providers-alibaba | apache-airflow-providers-amazon |                          +apache-airflow-providers-apache-beam | apache-airflow-providers-apache-cassandra |            +apache-airflow-providers-apache-drill | apache-airflow-providers-apache-druid |               +apache-airflow-providers-apache-hdfs | apache-airflow-providers-apache-hive |                 +apache-airflow-providers-apache-kylin | apache-airflow-providers-apache-livy |                +apache-airflow-providers-apache-pig | apache-airflow-providers-apache-pinot |                 +apache-airflow-providers-apache-spark | apache-airflow-providers-apache-sqoop |               +apache-airflow-providers-arangodb | apache-airflow-providers-asana |                          +apache-airflow-providers-celery | apache-airflow-providers-cloudant |                         +apache-airflow-providers-cncf-kubernetes | apache-airflow-providers-core-sql |                +apache-airflow-providers-databricks | apache-airflow-providers-datadog |                      +apache-airflow-providers-dbt-cloud | apache-airflow-providers-dingding |                      +apache-airflow-providers-discord | apache-airflow-providers-docker |                          +apache-airflow-providers-elasticsearch | apache-airflow-providers-exasol |                    +apache-airflow-providers-facebook | apache-airflow-providers-ftp |                            +apache-airflow-providers-github | apache-airflow-providers-google |                           +apache-airflow-providers-grpc | apache-airflow-providers-hashicorp |                          +apache-airflow-providers-http | apache-airflow-providers-imap |                               +apache-airflow-providers-influxdb | apache-airflow-providers-jdbc |                           +apache-airflow-providers-jenkins | apache-airflow-providers-jira |                            +apache-airflow-providers-microsoft-azure | apache-airflow-providers-microsoft-mssql |         +apache-airflow-providers-microsoft-psrp | apache-airflow-providers-microsoft-winrm |          +apache-airflow-providers-mongo | apache-airflow-providers-mysql |                             +apache-airflow-providers-neo4j | apache-airflow-providers-odbc |                              +apache-airflow-providers-openfaas | apache-airflow-providers-opsgenie |                       +apache-airflow-providers-oracle | apache-airflow-providers-pagerduty |                        +apache-airflow-providers-papermill | apache-airflow-providers-plexus |                        +apache-airflow-providers-postgres | apache-airflow-providers-presto |                         +apache-airflow-providers-qubole | apache-airflow-providers-redis |                            +apache-airflow-providers-salesforce | apache-airflow-providers-samba |                        +apache-airflow-providers-segment | apache-airflow-providers-sendgrid |                        +apache-airflow-providers-sftp | apache-airflow-providers-singularity |                        +apache-airflow-providers-slack | apache-airflow-providers-snowflake |                         +apache-airflow-providers-sqlite | apache-airflow-providers-ssh |                              +apache-airflow-providers-tableau | apache-airflow-providers-tabular |                         +apache-airflow-providers-telegram | apache-airflow-providers-trino |                          +apache-airflow-providers-vertica | apache-airflow-providers-yandex |                          +apache-airflow-providers-zendesk | docker-stack | helm-chart)                                 +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + + + diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt new file mode 100644 index 0000000000000..807430507b84b --- /dev/null +++ b/images/breeze/output-commands-hash.txt @@ -0,0 +1,37 @@ + +# This file is automatically generated by pre-commit. If you have a conflict with this file +# 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-image:b62509a59badf3aa230e4562df751002 +build-prod-image:1902ec077a6d70336de6038d13472ef3 +cleanup:9a94bd1063296ea86e895f671db0b330 +command-hash-export:83bc6a4a8c60b62da3d0f00e81d2c3ea +config:92653afc11889e1b78e3a2e38f41107f +docker-compose-tests:8ae3b6211fd31db81a750d1c6b96ec3d +exec:e4329909b8b2a610fa4fad5116c4b896 +find-newer-dependencies:00000f7afb289e36e8c573fcc654df44 +fix-ownership:596143cc74217f0a90850a554220ea45 +free-space:bb8e7ac63d12ab3ede272a898de2f527 +generate-constraints:a5120e79439f30eb7fbee929dca23156 +prepare-airflow-package:cff9d88ca313db10f3cc464c6798f6be +prepare-provider-documentation:95c864f8a656a95cac7d9c682cb75773 +prepare-provider-packages:33c0fe04ad4c6068b69ad1361b142057 +pull-image:a9bb83372b5da5212f48e2affeedc551 +pull-prod-image:6e8467a2b8c833a392c8bdd65189363e +regenerate-command-images:4fd2e7ecbfd6eebb18b854f3eb0f29c8 +release-prod-images:8858fe5a13989c7c65a79dc97a880928 +resource-check:0fb929ac3496dbbe97acfe99e35accd7 +selective-check:eb1cf022ae43fa9c737b1647142e5a96 +self-upgrade:b5437c0a1a91533a11ee9d0a9692369c +setup-autocomplete:355b72dee171c2fcba46fc90ac7c97b0 +shell:4680295fdd8a276d51518d29360c365c +start-airflow:92cf775a952439a32d409cd2536da507 +static-checks:c7adc5d6dff34624ef413d3d146ec974 +stop:8ebd8a42f1003495d37b884de5ac7ce6 +tests:ae8d62b505ff8f79bddc202fe9d575e3 +verify-image:a6b3c70957aea96a5d4d261f23359a2d +verify-prod-image:bf3cf39200e010e3015ef071fd387c6f +verify-provider-packages:797e60067fc4611112527de808b5c1c1 +version:d11da4c17a23179830079b646160149c diff --git a/images/breeze/output-prepare-provider-documentation.svg b/images/breeze/output-prepare-provider-documentation.svg new file mode 100644 index 0000000000000..6390fc20d9b88 --- /dev/null +++ b/images/breeze/output-prepare-provider-documentation.svg @@ -0,0 +1,168 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: prepare-provider-documentation + + + + + + + + + + +Usage: breeze prepare-provider-documentation [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra | +                                             apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin | +                                             apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop | +                                             arangodb | asana | celery | cloudant | cncf.kubernetes | core.sql | +                                             databricks | datadog | dbt.cloud | dingding | discord | docker | +                                             elasticsearch | exasol | facebook | ftp | github | google | grpc | +                                             hashicorp | http | imap | influxdb | jdbc | jenkins | jira | +                                             microsoft.azure | microsoft.mssql | microsoft.psrp | microsoft.winrm | +                                             mongo | mysql | neo4j | odbc | openfaas | opsgenie | oracle | pagerduty | +                                             papermill | plexus | postgres | presto | qubole | redis | salesforce | +                                             samba | segment | sendgrid | sftp | singularity | slack | snowflake | +                                             sqlite | ssh | tableau | tabular | telegram | trino | vertica | yandex | +                                             zendesk]... + +Prepare CHANGELOG, README and COMMITS information for providers. + +╭─ Provider documentation preparation flags ───────────────────────────────────────────────────────────────────────────╮ +--debugDrop user in shell instead of running the command. Useful for debugging. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + + + diff --git a/images/breeze/output-prepare-provider-packages.svg b/images/breeze/output-prepare-provider-packages.svg new file mode 100644 index 0000000000000..fe0c1733cbf59 --- /dev/null +++ b/images/breeze/output-prepare-provider-packages.svg @@ -0,0 +1,172 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: prepare-provider-packages + + + + + + + + + + +Usage: breeze prepare-provider-packages [OPTIONS] [airbyte | alibaba | amazon | apache.beam | apache.cassandra | +                                        apache.drill | apache.druid | apache.hdfs | apache.hive | apache.kylin | +                                        apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop | +                                        arangodb | asana | celery | cloudant | cncf.kubernetes | core.sql | databricks +                                        | datadog | dbt.cloud | dingding | discord | docker | elasticsearch | exasol | +                                        facebook | ftp | github | google | grpc | hashicorp | http | imap | influxdb | +                                        jdbc | jenkins | jira | microsoft.azure | microsoft.mssql | microsoft.psrp | +                                        microsoft.winrm | mongo | mysql | neo4j | odbc | openfaas | opsgenie | oracle +                                        | pagerduty | papermill | plexus | postgres | presto | qubole | redis | +                                        salesforce | samba | segment | sendgrid | sftp | singularity | slack | +                                        snowflake | sqlite | ssh | tableau | tabular | telegram | trino | vertica | +                                        yandex | zendesk]... + +Prepare sdist/whl packages of Airflow Providers. + +╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--package-formatFormat of packages.(wheel | sdist | both)[default: wheel] +--version-suffix-for-pypiVersion suffix used for PyPI packages (alpha, beta, rc1, etc.).(TEXT) +--package-list-fileRead list of packages from text file (one package per line)(FILENAME) +--debugDrop user in shell instead of running the command. Useful for debugging. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + + + From 287bf8c3c92e76f5ed1e2effee323f02139ef83f Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 5 Jul 2022 16:29:56 +0200 Subject: [PATCH 10/13] Move example dag --- airflow/providers/tabular/CHANGELOG.rst | 2 +- airflow/providers/tabular/provider.yaml | 2 +- .../example_dags => tests/system/providers/tabular}/__init__.py | 0 .../system/providers/tabular}/example_tabular.py | 0 4 files changed, 2 insertions(+), 2 deletions(-) rename {airflow/providers/tabular/example_dags => tests/system/providers/tabular}/__init__.py (100%) rename {airflow/providers/tabular/example_dags => tests/system/providers/tabular}/example_tabular.py (100%) diff --git a/airflow/providers/tabular/CHANGELOG.rst b/airflow/providers/tabular/CHANGELOG.rst index 1f10cb7ef7691..a152fd0836512 100644 --- a/airflow/providers/tabular/CHANGELOG.rst +++ b/airflow/providers/tabular/CHANGELOG.rst @@ -18,7 +18,7 @@ Changelog --------- -0.0.1 +1.0.0 ..... Initial version of the provider. diff --git a/airflow/providers/tabular/provider.yaml b/airflow/providers/tabular/provider.yaml index 6465b8e338d9e..02f6cb59e0e5c 100644 --- a/airflow/providers/tabular/provider.yaml +++ b/airflow/providers/tabular/provider.yaml @@ -22,7 +22,7 @@ description: | `Tabular `__ versions: - - 0.0.1 + - 1.0.0 dependencies: - apache-airflow>=2.2.0 diff --git a/airflow/providers/tabular/example_dags/__init__.py b/tests/system/providers/tabular/__init__.py similarity index 100% rename from airflow/providers/tabular/example_dags/__init__.py rename to tests/system/providers/tabular/__init__.py diff --git a/airflow/providers/tabular/example_dags/example_tabular.py b/tests/system/providers/tabular/example_tabular.py similarity index 100% rename from airflow/providers/tabular/example_dags/example_tabular.py rename to tests/system/providers/tabular/example_tabular.py From af93366889b625b468a73f13fe9e9898c008eb30 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 6 Jul 2022 08:20:31 +0200 Subject: [PATCH 11/13] Update docs --- docs/apache-airflow-providers-tabular/index.rst | 2 +- tests/system/providers/tabular/example_tabular.py | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/apache-airflow-providers-tabular/index.rst b/docs/apache-airflow-providers-tabular/index.rst index 21ebfe37da9b6..2f5084e4c798c 100644 --- a/docs/apache-airflow-providers-tabular/index.rst +++ b/docs/apache-airflow-providers-tabular/index.rst @@ -32,7 +32,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources Python API <_api/airflow/providers/tabular/index> diff --git a/tests/system/providers/tabular/example_tabular.py b/tests/system/providers/tabular/example_tabular.py index b8088bf76c59a..7021b0e8742ef 100644 --- a/tests/system/providers/tabular/example_tabular.py +++ b/tests/system/providers/tabular/example_tabular.py @@ -46,3 +46,9 @@ bash_command=bash_command, env={"TOKEN": TabularHook().get_token_macro()}, ) + + +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) From b963d20abb72933d44d4f92c3ce90e888051978b Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 6 Jul 2022 09:25:37 +0200 Subject: [PATCH 12/13] Add link --- docs/apache-airflow-providers-tabular/index.rst | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/apache-airflow-providers-tabular/index.rst b/docs/apache-airflow-providers-tabular/index.rst index 2f5084e4c798c..83c1e7bad7eca 100644 --- a/docs/apache-airflow-providers-tabular/index.rst +++ b/docs/apache-airflow-providers-tabular/index.rst @@ -37,6 +37,12 @@ Content Installing from sources Python API <_api/airflow/providers/tabular/index> +.. toctree:: + :hidden: + :caption: System tests + System Tests <_api/tests/system/providers/tabular/index> + + .. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME! From 7075b91304ada3b6d8b59925ff16572761c469e6 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 6 Jul 2022 12:56:33 +0200 Subject: [PATCH 13/13] Fix the doc --- docs/apache-airflow-providers-tabular/index.rst | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/docs/apache-airflow-providers-tabular/index.rst b/docs/apache-airflow-providers-tabular/index.rst index 83c1e7bad7eca..554ed4a5d9c7a 100644 --- a/docs/apache-airflow-providers-tabular/index.rst +++ b/docs/apache-airflow-providers-tabular/index.rst @@ -28,6 +28,12 @@ Content Connection types +.. toctree:: + :hidden: + :caption: System tests + + System Tests <_api/tests/system/providers/tabular/index> + .. toctree:: :maxdepth: 1 :caption: Resources @@ -37,11 +43,6 @@ Content Installing from sources Python API <_api/airflow/providers/tabular/index> -.. toctree:: - :hidden: - :caption: System tests - System Tests <_api/tests/system/providers/tabular/index> - .. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME!