From 29e6511be6cc6156a33622fe99777745029dbd5c Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 12 May 2022 21:54:10 +0200 Subject: [PATCH] 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