From 3bc1c5029ebb4ba4bde21f0cf1c60f407c9d3cd7 Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Mon, 22 Jan 2024 21:02:36 +0530 Subject: [PATCH 01/21] Initial Implementation of new Airflow Provider for Teradata cloud analytics and data platform Implemented new airflow provider with Hook, Operator and Transfer Operator --- .github/CODEOWNERS | 1 + .../airflow_providers_bug_report.yml | 1 + .github/boring-cyborg.yml | 6 + CONTRIBUTING.rst | 2 +- INSTALL | 3 +- airflow/providers/teradata/CHANGELOG.rst | 31 ++ airflow/providers/teradata/__init__.py | 44 +++ .../teradata/example_dags/__init__.py | 17 ++ .../example_dags/example_teradata_hook.py | 126 ++++++++ .../example_dags/example_teradata_operator.py | 132 +++++++++ ..._teradata_to_teradata_transfer_operator.py | 158 ++++++++++ airflow/providers/teradata/hooks/__init__.py | 17 ++ airflow/providers/teradata/hooks/teradata.py | 210 ++++++++++++++ .../providers/teradata/operators/__init__.py | 17 ++ .../providers/teradata/operators/teradata.py | 67 +++++ airflow/providers/teradata/provider.yaml | 61 ++++ .../providers/teradata/transfers/__init__.py | 16 + .../transfers/teradata_to_teradata.py | 91 ++++++ airflow/utils/db.py | 11 + ...release-management_add-back-references.svg | 2 +- ...ement_generate-issue-content-providers.svg | 2 +- ...agement_prepare-provider-documentation.svg | 2 +- ...e-management_prepare-provider-packages.svg | 2 +- ...t_sbom_generate-providers-requirements.svg | 2 +- ...t_sbom_generate-providers-requirements.txt | 2 +- .../changelog.rst | 19 ++ .../commits.rst | 26 ++ .../connections/teradata.rst | 82 ++++++ .../index.rst | 134 +++++++++ .../installing-providers-from-sources.rst | 18 ++ .../operators/index.rst | 25 ++ .../operators/teradata.rst | 36 +++ .../operators/teradata_to_teradata.rst | 38 +++ .../redirects.txt | 1 + .../security.rst | 38 +++ docs/apache-airflow/extra-packages-ref.rst | 2 + docs/integration-logos/teradata/Teradata.png | Bin 0 -> 2220 bytes docs/spelling_wordlist.txt | 4 + generated/provider_dependencies.json | 14 + images/breeze/output_build-docs.svg | 217 ++++++++++++++ images/breeze/output_build-docs.txt | 1 + ...release-management_add-back-references.txt | 1 + ...ement_generate-issue-content-providers.txt | 1 + ...agement_prepare-provider-documentation.txt | 1 + ...e-management_prepare-provider-packages.txt | 1 + ...output_release-management_publish-docs.svg | 227 +++++++++++++++ ...output_release-management_publish-docs.txt | 1 + pyproject.toml | 9 +- tests/providers/teradata/__init__.py | 16 + tests/providers/teradata/hooks/__init__.py | 17 ++ .../providers/teradata/hooks/test_teradata.py | 274 ++++++++++++++++++ .../providers/teradata/operators/__init__.py | 17 ++ .../teradata/operators/test_teradata.py | 128 ++++++++ .../providers/teradata/transfers/__init__.py | 17 ++ .../transfers/test_teradata_to_teradata.py | 71 +++++ tests/system/providers/teradata/__init__.py | 16 + .../providers/teradata/create_ssl_table.sql | 22 ++ .../providers/teradata/create_table.sql | 22 ++ .../teradata/example_ssl_teradata.py | 127 ++++++++ .../providers/teradata/example_teradata.py | 125 ++++++++ .../example_teradata_to_teradata_transfer.py | 179 ++++++++++++ 61 files changed, 2941 insertions(+), 9 deletions(-) create mode 100644 airflow/providers/teradata/CHANGELOG.rst create mode 100644 airflow/providers/teradata/__init__.py create mode 100644 airflow/providers/teradata/example_dags/__init__.py create mode 100644 airflow/providers/teradata/example_dags/example_teradata_hook.py create mode 100644 airflow/providers/teradata/example_dags/example_teradata_operator.py create mode 100644 airflow/providers/teradata/example_dags/example_teradata_to_teradata_transfer_operator.py create mode 100644 airflow/providers/teradata/hooks/__init__.py create mode 100644 airflow/providers/teradata/hooks/teradata.py create mode 100644 airflow/providers/teradata/operators/__init__.py create mode 100644 airflow/providers/teradata/operators/teradata.py create mode 100644 airflow/providers/teradata/provider.yaml create mode 100644 airflow/providers/teradata/transfers/__init__.py create mode 100644 airflow/providers/teradata/transfers/teradata_to_teradata.py create mode 100644 docs/apache-airflow-providers-teradata/changelog.rst create mode 100644 docs/apache-airflow-providers-teradata/commits.rst create mode 100644 docs/apache-airflow-providers-teradata/connections/teradata.rst create mode 100644 docs/apache-airflow-providers-teradata/index.rst create mode 100644 docs/apache-airflow-providers-teradata/installing-providers-from-sources.rst create mode 100644 docs/apache-airflow-providers-teradata/operators/index.rst create mode 100644 docs/apache-airflow-providers-teradata/operators/teradata.rst create mode 100644 docs/apache-airflow-providers-teradata/operators/teradata_to_teradata.rst create mode 100644 docs/apache-airflow-providers-teradata/redirects.txt create mode 100644 docs/apache-airflow-providers-teradata/security.rst create mode 100644 docs/integration-logos/teradata/Teradata.png create mode 100644 images/breeze/output_build-docs.svg create mode 100644 images/breeze/output_build-docs.txt create mode 100644 images/breeze/output_release-management_add-back-references.txt create mode 100644 images/breeze/output_release-management_generate-issue-content-providers.txt create mode 100644 images/breeze/output_release-management_prepare-provider-documentation.txt create mode 100644 images/breeze/output_release-management_prepare-provider-packages.txt create mode 100644 images/breeze/output_release-management_publish-docs.svg create mode 100644 images/breeze/output_release-management_publish-docs.txt create mode 100644 tests/providers/teradata/__init__.py create mode 100644 tests/providers/teradata/hooks/__init__.py create mode 100644 tests/providers/teradata/hooks/test_teradata.py create mode 100644 tests/providers/teradata/operators/__init__.py create mode 100644 tests/providers/teradata/operators/test_teradata.py create mode 100644 tests/providers/teradata/transfers/__init__.py create mode 100644 tests/providers/teradata/transfers/test_teradata_to_teradata.py create mode 100644 tests/system/providers/teradata/__init__.py create mode 100644 tests/system/providers/teradata/create_ssl_table.sql create mode 100644 tests/system/providers/teradata/create_table.sql create mode 100644 tests/system/providers/teradata/example_ssl_teradata.py create mode 100644 tests/system/providers/teradata/example_teradata.py create mode 100644 tests/system/providers/teradata/example_teradata_to_teradata_transfer.py diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index e58dee9e85fa44..230f385662c77f 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -75,6 +75,7 @@ airflow/providers/cncf/kubernetes/executors/ @dstandish @jedcunningham /tests/system/providers/amazon/ @eladkal @o-nikolas /tests/providers/common/sql/ @eladkal /tests/providers/slack/ @eladkal +/tests/providers/teradata/ @SatishChGit # Dev tools /.github/workflows/ @potiuk @ashb @kaxil diff --git a/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml b/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml index 3213bf8b3c2d15..736cff6227a173 100644 --- a/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml @@ -105,6 +105,7 @@ body: - tableau - tabular - telegram + - teradata - trino - vertica - weaviate diff --git a/.github/boring-cyborg.yml b/.github/boring-cyborg.yml index f6798b54dbd08d..382a992be4f5f4 100644 --- a/.github/boring-cyborg.yml +++ b/.github/boring-cyborg.yml @@ -486,6 +486,12 @@ labelPRBasedOnFilePath: - tests/providers/telegram/**/* - tests/system/providers/telegram/**/* + provider:teradata: + - airflow/providers/teradata/**/* + - docs/apache-airflow-providers-teradata/**/* + - tests/providers/teradata/**/* + - tests/system/providers/teradata/**/* + provider:trino: - airflow/providers/trino/**/* - docs/apache-airflow-providers-trino/**/* diff --git a/CONTRIBUTING.rst b/CONTRIBUTING.rst index 479ba765d44670..9be33bae34ddb5 100644 --- a/CONTRIBUTING.rst +++ b/CONTRIBUTING.rst @@ -900,7 +900,7 @@ microsoft-mssql, microsoft-psrp, microsoft-winrm, mongo, mssql, mysql, neo4j, od openfaas, openlineage, opensearch, opsgenie, oracle, otel, pagerduty, pandas, papermill, password, pgvector, pinecone, pinot, postgres, presto, rabbitmq, redis, s3, s3fs, salesforce, samba, saml, segment, sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd, -tableau, tabular, telegram, trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex, zendesk +tableau, tabular, telegram, teradata, trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex, zendesk .. END REGULAR EXTRAS HERE Provider packages diff --git a/INSTALL b/INSTALL index ab0ff03ef51787..46496967644636 100644 --- a/INSTALL +++ b/INSTALL @@ -254,7 +254,8 @@ microsoft-mssql, microsoft-psrp, microsoft-winrm, mongo, mssql, mysql, neo4j, od openfaas, openlineage, opensearch, opsgenie, oracle, otel, pagerduty, pandas, papermill, password, pgvector, pinecone, pinot, postgres, presto, rabbitmq, redis, s3, s3fs, salesforce, samba, saml, segment, sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd, -tableau, tabular, telegram, trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex, zendesk +tableau, tabular, telegram, teradata, trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex, +zendesk # END REGULAR EXTRAS HERE Devel extras - used to install development-related tools. Only available during editable install. diff --git a/airflow/providers/teradata/CHANGELOG.rst b/airflow/providers/teradata/CHANGELOG.rst new file mode 100644 index 00000000000000..46857d51a98134 --- /dev/null +++ b/airflow/providers/teradata/CHANGELOG.rst @@ -0,0 +1,31 @@ + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +.. NOTE TO CONTRIBUTORS: + Please, only add notes to the Changelog just below the "Changelog" header when there + are some breaking changes and you want to add an explanation to the users on how they are supposed + to deal with them. The changelog is updated and maintained semi-automatically by release manager. + +``apache-airflow-providers-teradata`` + +Changelog +--------- + +1.0.0 +..... + +Initial version of the provider. diff --git a/airflow/providers/teradata/__init__.py b/airflow/providers/teradata/__init__.py new file mode 100644 index 00000000000000..655822eebf7410 --- /dev/null +++ b/airflow/providers/teradata/__init__.py @@ -0,0 +1,44 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# +# NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE +# OVERWRITTEN WHEN PREPARING DOCUMENTATION FOR THE PACKAGES. +# +# IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE +# `PROVIDER__INIT__PY_TEMPLATE.py.jinja2` IN the `dev/provider_packages` DIRECTORY +# +from __future__ import annotations + +import packaging.version + +__all__ = ["__version__"] + +__version__ = "1.0.0" +MIN_AIRFLOW_VERSION = "2.6.0" + +try: + from airflow import __version__ as airflow_version +except ImportError: + from airflow.version import version as airflow_version + +if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( + MIN_AIRFLOW_VERSION +): + raise RuntimeError( + f"The package `apache-airflow-providers-teradata:{__version__}` requires Apache Airflow {MIN_AIRFLOW_VERSION}+" + ) diff --git a/airflow/providers/teradata/example_dags/__init__.py b/airflow/providers/teradata/example_dags/__init__.py new file mode 100644 index 00000000000000..217e5db9607827 --- /dev/null +++ b/airflow/providers/teradata/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/teradata/example_dags/example_teradata_hook.py b/airflow/providers/teradata/example_dags/example_teradata_hook.py new file mode 100644 index 00000000000000..f071076b8c5faa --- /dev/null +++ b/airflow/providers/teradata/example_dags/example_teradata_hook.py @@ -0,0 +1,126 @@ +# +# 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. +""" +Example Airflow DAG showing usage of Teradata Airflow Provider hook interacting with Teradata SQL DB. + +This DAG assumes Airflow Connection with connection id `teradata_default` already exists in locally. +It shows how to establish connection with Teradata SQL Database server and how to run queries on it. +""" +from __future__ import annotations + +from contextlib import closing +from datetime import datetime, timedelta + +import teradatasql + +from airflow import DAG +from airflow.operators.python import PythonOperator +from airflow.providers.teradata.hooks.teradata import TeradataHook + +# Constants used for below example DAG +TEST_SQL = "SELECT DATE;" +CONN_ID = "teradata_default" + +# [START howto_hook_teradata] + + +def teradata_hook_usage(): + # This creates connection with Teradata database using default connection id 'teradata_default' + tdh = TeradataHook() + + # This creates connection with Teradata database overriding database name from what is specified + # in teradata_default connection id + tdh = TeradataHook(teradata_conn_id=CONN_ID, database="database_name") + + # Verify connection to database server works + tdh.test_connection() + + # Check connection config details + tdh.get_uri() + # print(tdh.get_uri()) + + # This method gets back a TeradataConnection object which is created using teradatasql client internally + conn = tdh.get_conn() + + # This method returns sqlalchemy engine connection object that points to Teradata database + tdh.get_sqlalchemy_engine() + + # Execute select queries directly using TeradataHook + tdh.get_records(sql=TEST_SQL) + tdh.get_first(sql=TEST_SQL) + + # Execute any other modification queries using TeradataHook + tdh.run(sql=TEST_SQL) + tdh.run(sql=["SELECT 1;", "SELECT 2;"]) + + # Execute insert queries + # rows = [('User5',), ('User6',)] + # target_fields = ["user_name"] + # res = tdh.insert_rows(table="my_users", rows=rows, target_fields=target_fields) + + # Get cursor object from connection to manually run queries and get results + # Read more about using teradatasql connection here: https://pypi.org/project/teradatasql/ + cursor = tdh.get_cursor() + cursor.execute(TEST_SQL).fetchall() + [d[0] for d in cursor.description] + try: + # print("Test executing queries in Teradata database...") + with closing(tdh.get_conn()) as conn, closing(conn.cursor()) as cursor: + cursor.execute("SELECT DATE") + conn.commit() + print("Ran the query on Teradata database") + except (teradatasql.OperationalError, teradatasql.Warning): + # print("Error running query on Teradata database") + raise + + # Get query result as pandas dataframe + tdh.get_pandas_df(sql=TEST_SQL) + + # Get query results as chunks of rows as pandas generator + gen = tdh.get_pandas_df_by_chunks(sql=TEST_SQL, chunksize=2) + while True: + try: + next(gen) + except StopIteration: + break + # print(rows) + + # Saving data to a staging table using pandas to_sql + # conn = tdh.get_sqlalchemy_engine() + # df.to_sql("temp_my_users", con=conn, if_exists="replace") + + +# [END howto_hook_teradata] + + +with DAG( + dag_id="example_teradata_hook", + description="""Sample usage of the TeradataHook airflow provider module""", + start_date=datetime(2023, 1, 1), + max_active_runs=1, + max_active_tasks=3, + catchup=False, + tags=["example"], + default_args={ + "retries": 2, + "retry_delay": timedelta(minutes=6), + }, +) as dag: + show_teradata_hook_usage = PythonOperator( + task_id="show_teradata_hook_usage", python_callable=teradata_hook_usage + ) diff --git a/airflow/providers/teradata/example_dags/example_teradata_operator.py b/airflow/providers/teradata/example_dags/example_teradata_operator.py new file mode 100644 index 00000000000000..6b868c1bda06e3 --- /dev/null +++ b/airflow/providers/teradata/example_dags/example_teradata_operator.py @@ -0,0 +1,132 @@ +# +# 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. +""" +Example Airflow DAG to show basic CRUD operation on teradata database using TeradataOperator + +This DAG assumes Airflow Connection with connection id `teradata_default` already exists in locally. +It shows how to run queries as tasks in airflow dags using TeradataOperator.. +""" +from __future__ import annotations + +from datetime import datetime + +import pytest + +from airflow import DAG +from airflow.models.baseoperator import chain + +try: + from airflow.providers.teradata.operators.teradata import TeradataOperator +except ImportError: + pytest.skip("Teradata provider apache-airflow-provider-teradata not available", allow_module_level=True) + + +CONN_ID = "teradata_default" + + +with DAG( + dag_id="example_teradata_operator", + max_active_runs=1, + max_active_tasks=3, + catchup=False, + start_date=datetime(2023, 1, 1), +) as dag: + # [START howto_teradata_operator] + + create = TeradataOperator( + task_id="table_create", + conn_id=CONN_ID, + sql=""" + CREATE TABLE my_users, + FALLBACK ( + user_id decimal(10,0) NOT NULL GENERATED ALWAYS AS IDENTITY ( + START WITH 1 + INCREMENT BY 1 + MINVALUE 1 + MAXVALUE 2147483647 + NO CYCLE), + user_name VARCHAR(30) + ) PRIMARY INDEX (user_id); + """, + ) + + insert = TeradataOperator( + task_id="insert_rows", + conn_id=CONN_ID, + sql=""" + INSERT INTO my_users(user_name) VALUES ('User1'); + INSERT INTO my_users(user_name) VALUES ('User2'); + INSERT INTO my_users(user_name) VALUES ('User3'); + INSERT INTO my_users(user_name) VALUES ('User4'); + INSERT INTO my_users(user_name) VALUES ('User5'); + INSERT INTO my_users(user_name) VALUES ('User6'); + INSERT INTO my_users(user_name) VALUES ('User7'); + INSERT INTO my_users(user_name) VALUES ('User8'); + INSERT INTO my_users(user_name) VALUES ('User9'); + INSERT INTO my_users(user_name) VALUES ('User10'); + """, + ) + + update = TeradataOperator( + task_id="update_data", + conn_id=CONN_ID, + sql=""" + UPDATE my_users SET user_name = 'User123' WHERE user_name = 'User1'; + """, + ) + + read_data = TeradataOperator( + task_id="read_data", + conn_id=CONN_ID, + sql=""" + SELECT TOP 10 * from my_users order by user_id desc; + """, + ) + + delete = TeradataOperator( + task_id="delete_rows", + conn_id=CONN_ID, + sql=""" + DELETE FROM my_users ALL; + """, + ) + + alter_table = TeradataOperator( + task_id="alter_table", + conn_id=CONN_ID, + sql=""" + ALTER TABLE my_users + ADD gender CHAR(1) DEFAULT 'M', + ADD birth_date DATE FORMAT 'YYYY-MM-DD' NOT NULL DEFAULT DATE '2023-01-01'; + """, + ) + + drop = TeradataOperator( + task_id="drop_table", + conn_id=CONN_ID, + sql=""" + DROP TABLE my_users; + """, + ) + + chain(create, insert, update, read_data, delete, alter_table, drop) + + # Make sure create was done before deleting table + create >> drop + + # [END howto_teradata_operator] diff --git a/airflow/providers/teradata/example_dags/example_teradata_to_teradata_transfer_operator.py b/airflow/providers/teradata/example_dags/example_teradata_to_teradata_transfer_operator.py new file mode 100644 index 00000000000000..49c97485dcf715 --- /dev/null +++ b/airflow/providers/teradata/example_dags/example_teradata_to_teradata_transfer_operator.py @@ -0,0 +1,158 @@ +# +# 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. +""" +Example Airflow DAG to show usage of teradata to teradata transfer operator + +The transfer operator connects to source teradata server, runs query to fetch data from source +and inserts that data into destination teradata database server. It assumes tables already exists. +The example DAG below assumes Airflow Connection with connection id `teradata_default` already exists. +It creates sample my_users table at source and destination, sets up sample data at source and then +runs transfer operator to copy data to corresponding table on destination server. +""" +from __future__ import annotations + +from datetime import datetime + +import pytest + +from airflow import DAG +from airflow.models.baseoperator import chain + +try: + from airflow.providers.teradata.operators.teradata import TeradataOperator + from airflow.providers.teradata.transfers.teradata_to_teradata import TeradataToTeradataOperator +except ImportError: + pytest.skip("Teradata provider apache-airflow-provider-teradata not available", allow_module_level=True) + + +CONN_ID = "teradata_default" + + +with DAG( + dag_id="example_teradata_to_teradata_transfer_operator", + max_active_runs=1, + max_active_tasks=3, + catchup=False, + start_date=datetime(2023, 1, 1), +) as dag: + # [START howto_transfer_operator_teradata_to_teradata] + + create_src_table = TeradataOperator( + task_id="create_src_table", + conn_id=CONN_ID, + sql=""" + CREATE TABLE my_users_src, + FALLBACK ( + user_id decimal(10,0) NOT NULL GENERATED ALWAYS AS IDENTITY ( + START WITH 1 + INCREMENT BY 1 + MINVALUE 1 + MAXVALUE 2147483647 + NO CYCLE), + user_name VARCHAR(30), + gender CHAR(1) DEFAULT 'M', + birth_date DATE FORMAT 'YYYY-MM-DD' NOT NULL DEFAULT DATE '2023-01-01' + ) PRIMARY INDEX (user_id); + """, + ) + + create_dest_table = TeradataOperator( + task_id="create_dest_table", + conn_id=CONN_ID, + sql=""" + CREATE TABLE my_users_dest, + FALLBACK ( + user_id decimal(10,0) NOT NULL GENERATED ALWAYS AS IDENTITY ( + START WITH 1 + INCREMENT BY 1 + MINVALUE 1 + MAXVALUE 2147483647 + NO CYCLE), + user_name VARCHAR(30), + gender CHAR(1) DEFAULT 'M', + birth_date DATE FORMAT 'YYYY-MM-DD' NOT NULL DEFAULT DATE '2023-01-01' + ) PRIMARY INDEX (user_id); + """, + ) + insert_data_src = TeradataOperator( + task_id="insert_data_src", + conn_id=CONN_ID, + sql=""" + INSERT INTO my_users_src(user_name) VALUES ('User1'); + INSERT INTO my_users_src(user_name) VALUES ('User2'); + INSERT INTO my_users_src(user_name) VALUES ('User3'); + """, + ) + + read_data_src = TeradataOperator( + task_id="read_data_src", + conn_id=CONN_ID, + sql=""" + SELECT TOP 10 * from my_users_src order by user_id desc; + """, + ) + + transfer_data = TeradataToTeradataOperator( + task_id="transfer_data", + teradata_destination_conn_id="teradata_default", + destination_table="my_users_dest", + teradata_source_conn_id="teradata_default", + source_sql="select * from my_users_src", + source_sql_params={}, + rows_chunk=2, + ) + + read_data_dest = TeradataOperator( + task_id="read_data_dest", + conn_id=CONN_ID, + sql=""" + SELECT TOP 10 * from my_users_dest order by user_id desc; + """, + ) + + drop_src_table = TeradataOperator( + task_id="drop_src_table", + conn_id=CONN_ID, + sql=""" + DROP TABLE my_users_src; + """, + ) + + drop_dest_table = TeradataOperator( + task_id="drop_dest_table", + conn_id=CONN_ID, + sql=""" + DROP TABLE my_users_dest; + """, + ) + + chain( + create_src_table, + create_dest_table, + insert_data_src, + read_data_src, + transfer_data, + read_data_dest, + drop_src_table, + drop_dest_table, + ) + + # Make sure create was done before deleting table + create_src_table >> drop_src_table + create_dest_table >> drop_dest_table + # [END howto_transfer_operator_teradata_to_teradata] diff --git a/airflow/providers/teradata/hooks/__init__.py b/airflow/providers/teradata/hooks/__init__.py new file mode 100644 index 00000000000000..217e5db9607827 --- /dev/null +++ b/airflow/providers/teradata/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/teradata/hooks/teradata.py b/airflow/providers/teradata/hooks/teradata.py new file mode 100644 index 00000000000000..a53f1e4b71f2eb --- /dev/null +++ b/airflow/providers/teradata/hooks/teradata.py @@ -0,0 +1,210 @@ +# +# 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. +"""A Airflow Hook for interacting with Teradata SQL Server.""" +from __future__ import annotations + +from typing import TYPE_CHECKING, Any, TypeVar + +import sqlalchemy +import teradatasql +from teradatasql import TeradataConnection + +from airflow.providers.common.sql.hooks.sql import DbApiHook + +T = TypeVar("T") +if TYPE_CHECKING: + from airflow.models.connection import Connection + + +class TeradataHook(DbApiHook): + """General hook for interacting with Teradata SQL Database. + + This module contains basic APIs to connect to and interact with Teradata SQL Database. It uses teradatasql + client internally as a database driver for connecting to Teradata database. The config parameters like + Teradata DB Server URL, username, password and database name are fetched from the predefined connection + config connection_id. It raises an airflow error if the given connection id doesn't exist. + + See :doc:` docs/apache-airflow-providers-teradata/connections/teradata.rst` for full documentation. + + :param args: passed to DbApiHook + :param kwargs: passed to DbApiHook + + + Usage Help: + + >>> tdh = TeradataHook() + >>> sql = "SELECT top 1 _airbyte_ab_id from airbyte_td._airbyte_raw_Sales;" + >>> tdh.get_records(sql) + [[61ad1d63-3efd-4da4-9904-a4489cc3a520]] + + """ + + # Override to provide the connection name. + conn_name_attr = "teradata_conn_id" + + # Override to have a default connection id for a particular dbHook + default_conn_name = "teradata_default" + + # Override if this db supports autocommit. + supports_autocommit = True + + # Override this for hook to have a custom name in the UI selection + conn_type = "teradata" + + # Override hook name to give descriptive name for hook + hook_name = "Teradata" + + # Override with the Teradata specific placeholder parameter string used for insert queries + placeholder: str = "?" + + # Override SQL query to be used for testing database connection + _test_connection_sql = "select 1" + + def __init__( + self, + *args, + database: str | None = None, + **kwargs, + ) -> None: + super().__init__(*args, schema=database, **kwargs) + + def get_conn(self) -> TeradataConnection: + """Creates and returns a Teradata Connection object using teradatasql client. + + Establishes connection to a Teradata SQL database using config corresponding to teradata_conn_id. + + .. note:: By default it connects to the database via the teradatasql library. + But you can also choose the mysql-connector-python library which lets you connect through ssl + without any further ssl parameters required. + + :return: a mysql connection object + """ + teradata_conn_config: dict = self._get_conn_config_teradatasql() + teradata_conn = teradatasql.connect(**teradata_conn_config) + return teradata_conn + + def bulk_insert_rows( + self, + table: str, + rows: list[tuple], + target_fields: list[str] | None = None, + commit_every: int = 5000, + ): + """A bulk insert of records for Teradata SQL Database. + + This uses prepared statements via `executemany()`. For best performance, + pass in `rows` as an iterator. + + :param table: target Teradata database table, use dot notation to target a + specific database + :param rows: the rows to insert into the table + :param target_fields: the names of the columns to fill in the table, default None. + If None, each rows should have some order as table columns name + :param commit_every: the maximum number of rows to insert in one transaction + Default 5000. Set greater than 0. Set 1 to insert each row in each transaction + """ + if not rows: + raise ValueError("parameter rows could not be None or empty iterable") + conn = self.get_conn() + if self.supports_autocommit: + self.set_autocommit(conn, False) + cursor = conn.cursor() + cursor.fast_executemany = True + values_base = target_fields if target_fields else rows[0] + prepared_stm = "INSERT INTO {tablename} {columns} VALUES ({values})".format( + tablename=table, + columns="({})".format(", ".join(target_fields)) if target_fields else "", + values=", ".join("?" for i in range(1, len(values_base) + 1)), + ) + row_count = 0 + # Chunk the rows + row_chunk = [] + for row in rows: + row_chunk.append(row) + row_count += 1 + if row_count % commit_every == 0: + cursor.executemany(prepared_stm, row_chunk) + conn.commit() # type: ignore[attr-defined] + # Empty chunk + row_chunk = [] + # Commit the leftover chunk + if len(row_chunk) > 0: + cursor.executemany(prepared_stm, row_chunk) + conn.commit() # type: ignore[attr-defined] + self.log.info("[%s] inserted %s rows", table, row_count) + cursor.close() + conn.close() # type: ignore[attr-defined] + + def _get_conn_config_teradatasql(self) -> dict[str, Any]: + """Returns set of config params required for connecting to Teradata DB using teradatasql client.""" + conn: Connection = self.get_connection(getattr(self, self.conn_name_attr)) + conn_config = { + "host": conn.host or "localhost", + "dbs_port": conn.port or "1025", + "database": conn.schema or "", + "user": conn.login or "dbc", + "password": conn.password or "dbc", + } + + if conn.extra_dejson.get("tmode", False): + conn_config["tmode"] = conn.extra_dejson["tmode"] + + # Handling SSL connection parameters + + if conn.extra_dejson.get("sslmode", False): + conn_config["sslmode"] = conn.extra_dejson["sslmode"] + if "verify" in conn_config["sslmode"]: + if conn.extra_dejson.get("sslca", False): + conn_config["sslca"] = conn.extra_dejson["sslca"] + if conn.extra_dejson.get("sslcapath", False): + conn_config["sslcapath"] = conn.extra_dejson["sslcapath"] + if conn.extra_dejson.get("sslcipher", False): + conn_config["sslcipher"] = conn.extra_dejson["sslcipher"] + if conn.extra_dejson.get("sslcrc", False): + conn_config["sslcrc"] = conn.extra_dejson["sslcrc"] + if conn.extra_dejson.get("sslprotocol", False): + conn_config["sslprotocol"] = conn.extra_dejson["sslprotocol"] + + return conn_config + + def get_sqlalchemy_engine(self, engine_kwargs=None): + """Returns a connection object using sqlalchemy.""" + conn: Connection = self.get_connection(getattr(self, self.conn_name_attr)) + link = f"teradatasql://{conn.login}:{conn.password}@{conn.host}" + connection = sqlalchemy.create_engine(link) + return connection + + @staticmethod + def get_ui_field_behaviour() -> dict: + """Returns custom field behaviour.""" + import json + + return { + "hidden_fields": ["port"], + "relabeling": { + "host": "Database Server URL", + "schema": "Database Name", + "login": "Username", + "password": "Password", + }, + "placeholders": { + "extra": json.dumps({"example_parameter": "parameter"}, indent=4), + "login": "dbc", + "password": "dbc", + }, + } diff --git a/airflow/providers/teradata/operators/__init__.py b/airflow/providers/teradata/operators/__init__.py new file mode 100644 index 00000000000000..217e5db9607827 --- /dev/null +++ b/airflow/providers/teradata/operators/__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/teradata/operators/teradata.py b/airflow/providers/teradata/operators/teradata.py new file mode 100644 index 00000000000000..8a496768fc7a4b --- /dev/null +++ b/airflow/providers/teradata/operators/teradata.py @@ -0,0 +1,67 @@ +# +# 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 __future__ import annotations + +from typing import Sequence + +from airflow.providers.common.sql.operators.sql import SQLExecuteQueryOperator +from airflow.providers.teradata.hooks.teradata import TeradataHook + + +class TeradataOperator(SQLExecuteQueryOperator): + """ + General Teradata Operator to execute queries on Teradata Database. + + Executes sql statements in the Teradata SQL Database using teradatasql jdbc driver + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:TeradataOperator` + :param sql: the SQL query to be executed as a single string, or a list of str (sql statements) + :param conn_id: reference to a predefined database + :param autocommit: if True, each command is automatically committed.(default value: False) + :param parameters: (optional) the parameters to render the SQL query with. + """ + + template_fields: Sequence[str] = ( + "parameters", + "sql", + ) + template_ext: Sequence[str] = (".sql",) + template_fields_renderers = {"sql": "sql"} + ui_color = "#e07c24" + + def __init__( + self, + conn_id: str = TeradataHook.default_conn_name, + host: str | None = None, + schema: str | None = None, + login: str | None = None, + password: str | None = None, + **kwargs, + ) -> None: + if any([host, schema, login, password]): + hook_params = kwargs.pop("hook_params", {}) + kwargs["hook_params"] = { + "host": host, + "schema": schema, + "login": login, + "password": password, + **hook_params, + } + super().__init__(**kwargs) + self.conn_id = conn_id diff --git a/airflow/providers/teradata/provider.yaml b/airflow/providers/teradata/provider.yaml new file mode 100644 index 00000000000000..f3d4b275c5ee1c --- /dev/null +++ b/airflow/providers/teradata/provider.yaml @@ -0,0 +1,61 @@ +# 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-teradata +name: Teradata +description: | + `Teradata `__ + +state: ready +source-date-epoch: 1700148082 +versions: + - 1.0.0 + +dependencies: + - apache-airflow>=2.6.0 + - apache-airflow-providers-common-sql>=1.3.1 + - teradatasqlalchemy>=17.20.0.0 + - teradatasql>=17.20.0.28 + +integrations: + - integration-name: Teradata + external-doc-url: https://www.teradata.com/ + how-to-guide: + - /docs/apache-airflow-providers-teradata/operators/teradata.rst + logo: /integration-logos/teradata/Teradata.png + tags: [software] + +operators: + - integration-name: Teradata + python-modules: + - airflow.providers.teradata.operators.teradata + +hooks: + - integration-name: Teradata + python-modules: + - airflow.providers.teradata.hooks.teradata + +transfers: + - source-integration-name: Teradata + target-integration-name: Teradata + python-module: airflow.providers.teradata.transfers.teradata_to_teradata + how-to-guide: /docs/apache-airflow-providers-teradata/operators/teradata_to_teradata.rst + +connection-types: + - hook-class-name: airflow.providers.teradata.hooks.teradata.TeradataHook + connection-type: teradata diff --git a/airflow/providers/teradata/transfers/__init__.py b/airflow/providers/teradata/transfers/__init__.py new file mode 100644 index 00000000000000..13a83393a9124b --- /dev/null +++ b/airflow/providers/teradata/transfers/__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/airflow/providers/teradata/transfers/teradata_to_teradata.py b/airflow/providers/teradata/transfers/teradata_to_teradata.py new file mode 100644 index 00000000000000..2ff491482895a5 --- /dev/null +++ b/airflow/providers/teradata/transfers/teradata_to_teradata.py @@ -0,0 +1,91 @@ +# +# 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 __future__ import annotations + +from typing import TYPE_CHECKING, Sequence + +from airflow.models import BaseOperator +from airflow.providers.teradata.hooks.teradata import TeradataHook + +if TYPE_CHECKING: + from airflow.utils.context import Context + + +class TeradataToTeradataOperator(BaseOperator): + """ + Moves data from Teradata source database to Teradata destination database. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:TeradataToTeradataOperator` + + :param teradata_destination_conn_id: destination Teradata connection. + :param destination_table: destination table to insert rows. + :param teradata_source_conn_id: :ref:`Source Teradata connection `. + :param source_sql: SQL query to execute against the source Teradata database + :param source_sql_params: Parameters to use in sql query. + :param rows_chunk: number of rows per chunk to commit. + """ + + template_fields: Sequence[str] = ( + "source_sql", + "source_sql_params", + ) + template_ext: Sequence[str] = (".sql",) + template_fields_renderers = {"source_sql": "sql", "source_sql_params": "py"} + ui_color = "#e07c24" + + def __init__( + self, + *, + teradata_destination_conn_id: str, + destination_table: str, + teradata_source_conn_id: str, + source_sql: str, + source_sql_params: dict | None = None, + rows_chunk: int = 5000, + **kwargs, + ) -> None: + super().__init__(**kwargs) + if source_sql_params is None: + source_sql_params = {} + self.teradata_destination_conn_id = teradata_destination_conn_id + self.destination_table = destination_table + self.teradata_source_conn_id = teradata_source_conn_id + self.source_sql = source_sql + self.source_sql_params = source_sql_params + self.rows_chunk = rows_chunk + + def _execute(self, src_hook, dest_hook, context) -> None: + with src_hook.get_conn() as src_conn: + cursor = src_conn.cursor() + cursor.execute(self.source_sql, self.source_sql_params) + target_fields = [field[0] for field in cursor.description] + rows_total = 0 + for rows in iter(lambda: cursor.fetchmany(self.rows_chunk), []): + dest_hook.bulk_insert_rows( + self.destination_table, rows, target_fields=target_fields, commit_every=self.rows_chunk + ) + rows_total += len(rows) + self.log.info("Finished data transfer.") + cursor.close() + + def execute(self, context: Context) -> None: + src_hook = TeradataHook(teradata_conn_id=self.teradata_source_conn_id) + dest_hook = TeradataHook(teradata_conn_id=self.teradata_destination_conn_id) + self._execute(src_hook, dest_hook, context) diff --git a/airflow/utils/db.py b/airflow/utils/db.py index b9ee8323611c4c..08f9e854f99939 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -640,6 +640,17 @@ def create_default_connections(session: Session = NEW_SESSION): ), session, ) + merge_conn( + Connection( + conn_id="teradata_default", + conn_type="teradata", + host="localhost", + login="user", + password="password", + schema="schema", + ), + session, + ) merge_conn( Connection( conn_id="trino_default", diff --git a/dev/breeze/doc/images/output_release-management_add-back-references.svg b/dev/breeze/doc/images/output_release-management_add-back-references.svg index a13066ce0d945b..e6da6227e846be 100644 --- a/dev/breeze/doc/images/output_release-management_add-back-references.svg +++ b/dev/breeze/doc/images/output_release-management_add-back-references.svg @@ -146,7 +146,7 @@ microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage | opensearch |       opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | redis | salesforce | samba |     segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular | telegram |     -trino | vertica | weaviate | yandex | zendesk]...                                                                      +teradata | trino | vertica | weaviate | yandex | zendesk]...                                                           Command to add back references for documentation to make it backward compatible. diff --git a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg index e8e193e1d1e6ba..041d6526341677 100644 --- a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg +++ b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg @@ -146,7 +146,7 @@ microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage | opensearch |     opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | redis | salesforce | samba |     segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular | telegram |     -trino | vertica | weaviate | yandex | zendesk]...                                                                      +teradata | trino | vertica | weaviate | yandex | zendesk]...                                                           Generates content for issue to test the release. diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg index 561b4a627d678b..68849ea5a9a388 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg @@ -182,7 +182,7 @@ microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage | opensearch |     opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | redis | salesforce | samba |     segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular | telegram |     -trino | vertica | weaviate | yandex | zendesk]...                                                                      +teradata | trino | vertica | weaviate | yandex | zendesk]...                                                           Prepare CHANGELOG, README and COMMITS information for providers. diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-packages.svg b/dev/breeze/doc/images/output_release-management_prepare-provider-packages.svg index 970ef409242933..c532f2a5744a9e 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-packages.svg +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-packages.svg @@ -167,7 +167,7 @@ microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage | opensearch |     opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | redis | salesforce | samba |     segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular | telegram |     -trino | vertica | weaviate | yandex | zendesk]...                                                                      +teradata | trino | vertica | weaviate | yandex | zendesk]...                                                           Prepare sdist/whl packages of Airflow Providers. diff --git a/dev/breeze/doc/images/output_sbom_generate-providers-requirements.svg b/dev/breeze/doc/images/output_sbom_generate-providers-requirements.svg index dbc947ad26b4ab..1aeb0530aae9b5 100644 --- a/dev/breeze/doc/images/output_sbom_generate-providers-requirements.svg +++ b/dev/breeze/doc/images/output_sbom_generate-providers-requirements.svg @@ -193,7 +193,7 @@ │| neo4j | odbc | openai | openfaas | openlineage | opensearch | opsgenie | oracle | pagerduty â”‚ │| papermill | pgvector | pinecone | postgres | presto | redis | salesforce | samba | segment |│ │sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular | â”‚ -│telegram | trino | vertica | weaviate | yandex | zendesk)                                     â”‚ +│telegram | teradata | trino | vertica | weaviate | yandex | zendesk)                          â”‚ │--provider-versionProvider version to generate the requirements for i.e `2.1.0`. `latest` is also a supported   â”‚ │value to account for the most recent version of the provider                                  â”‚ │(TEXT)                                                                                        â”‚ diff --git a/dev/breeze/doc/images/output_sbom_generate-providers-requirements.txt b/dev/breeze/doc/images/output_sbom_generate-providers-requirements.txt index 56eadeec24b79a..e5d81cc64843cc 100644 --- a/dev/breeze/doc/images/output_sbom_generate-providers-requirements.txt +++ b/dev/breeze/doc/images/output_sbom_generate-providers-requirements.txt @@ -1 +1 @@ -84d46887b3f47bc209014ec5cb26406c +5255746965f918f6b3a35ac1b0806cb3 diff --git a/docs/apache-airflow-providers-teradata/changelog.rst b/docs/apache-airflow-providers-teradata/changelog.rst new file mode 100644 index 00000000000000..da5e9aa653205e --- /dev/null +++ b/docs/apache-airflow-providers-teradata/changelog.rst @@ -0,0 +1,19 @@ + + .. 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:: ../../airflow/providers/teradata/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-teradata/commits.rst b/docs/apache-airflow-providers-teradata/commits.rst new file mode 100644 index 00000000000000..5ba943611abcc6 --- /dev/null +++ b/docs/apache-airflow-providers-teradata/commits.rst @@ -0,0 +1,26 @@ + .. 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-teradata +------------------------------------------------------ + +`Teradata `__ + + +This is detailed commit list of changes for versions provider package: ``teradata``. +For high-level changelog, see :doc:`package information including changelog `. diff --git a/docs/apache-airflow-providers-teradata/connections/teradata.rst b/docs/apache-airflow-providers-teradata/connections/teradata.rst new file mode 100644 index 00000000000000..01e11fa653e4b8 --- /dev/null +++ b/docs/apache-airflow-providers-teradata/connections/teradata.rst @@ -0,0 +1,82 @@ + .. 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:teradata: + +Teradata Connection +====================== +The Teradata connection type enables integrations with Teradata. + +Configuring the Connection +-------------------------- +Host (required) + The host to connect to. + +Database (optional) + Specify the name of the database to connect to. + +Login (required) + Specify the user name to connect. + +Password (required) + Specify the password to connect. + +Extra (optional) + Specify the extra parameters (as json dictionary) that can be used in Teradata + connection. The following parameters out of the standard python parameters + are supported: + + * ``tmode`` - Specifies the transaction mode.Possible values are DEFAULT (the default), ANSI, or TERA + * ``sslmode`` - This option specifies the mode for connections to the database. + There are six modes: + ``disable``, ``allow``, ``prefer``, ``require``, ``verify-ca``, ``verify-full``. + * ``sslca`` - This parameter specifies the file name of a PEM file that contains + Certificate Authority (CA) certificates for use with sslmode values VERIFY-CA or VERIFY-FULL. + * ``sslcapath`` - This parameter specifies the TLS cipher for HTTPS/TLS connections. + * ``sslcipher`` - This parameter specifies the name of a file containing SSL + certificate authority (CA) certificate(s). + * ``sslcrc`` - This parameter controls TLS certificate revocation checking for + HTTPS/TLS connections when sslmode is VERIFY-FULL. + * ``sslprotocol`` - Specifies the TLS protocol for HTTPS/TLS connections. + + More details on all Teradata parameters supported can be found in + `Teradata documentation `_. + + Example "extras" field: + + .. code-block:: json + + { + "tmode": "TERA", + "sslmode": "verify-ca", + "sslcert": "/tmp/client-cert.pem", + "sslca": "/tmp/server-ca.pem", + "sslkey": "/tmp/client-key.pem" + } + + + When specifying the connection as URI (in :envvar:`AIRFLOW_CONN_{CONN_ID}` variable) you should specify it + following the standard syntax of DB connections, where extras are passed as parameters + of the URI (note that all components of the URI should be URL-encoded). + + For example: + + .. code-block:: bash + + export AIRFLOW_CONN_TERADATA_DEFAULT='teradata://teradata_user:XXXXXXXXXXXX@1.1.1.1:/teradatadb?tmode=tera&sslmode=verify-ca&sslca=%2Ftmp%2Fserver-ca.pem' diff --git a/docs/apache-airflow-providers-teradata/index.rst b/docs/apache-airflow-providers-teradata/index.rst new file mode 100644 index 00000000000000..13dd7a4e9b79e2 --- /dev/null +++ b/docs/apache-airflow-providers-teradata/index.rst @@ -0,0 +1,134 @@ + + .. 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-teradata`` +===================================== + + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: Basics + + Home + Changelog + Security + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: Guides + + Connection types + Operators + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: References + + Python API <_api/airflow/providers/teradata/index> + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: System tests + + System Tests <_api/tests/system/providers/teradata/index> + System Tests Dashboard + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: Resources + + Example DAGs + PyPI Repository + Installing from sources + +.. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME! + + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: Commits + + Detailed list of commits + + +Package apache-airflow-providers-teradata +------------------------------------------------------ + +`Teradata `__ + + +Release: 1.0.0 + +Provider package +---------------- + +This is a provider package for ``teradata`` provider. All classes for this provider package +are in ``airflow.providers.teradata`` python package. + +Installation +------------ + +You can install this package on top of an existing Airflow 2 installation (see ``Requirements`` below) +for the minimum Airflow version supported) via +``pip install apache-airflow-providers-teradata`` + +Requirements +------------ + +The minimum Apache Airflow version supported by this provider package is ``2.6.0``. + +======================================= ================== +PIP package Version required +======================================= ================== +``apache-airflow`` ``>=2.6.0`` +``apache-airflow-providers-common-sql`` ``>=1.3.1`` +======================================= ================== + +Cross provider package dependencies +----------------------------------- + +Those are dependencies that might be needed in order to use all the features of the package. +You need to install the specified provider packages in order to use them. + +You can install such cross-provider dependencies when installing from PyPI. For example: + +.. code-block:: bash + + pip install apache-airflow-providers-teradata[common.sql] + + +============================================================================================================ ============== +Dependent package Extra +============================================================================================================ ============== +`apache-airflow-providers-common-sql `_ ``common.sql`` +============================================================================================================ ============== + +Downloading official packages +----------------------------- + +You can download officially released packages and verify their checksums and signatures from the +`Official Apache Download site `_ + +* `The apache-airflow-providers-teradata 1.0.0 sdist package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-teradata 1.0.0 wheel package `_ (`asc `__, `sha512 `__) diff --git a/docs/apache-airflow-providers-teradata/installing-providers-from-sources.rst b/docs/apache-airflow-providers-teradata/installing-providers-from-sources.rst new file mode 100644 index 00000000000000..b4e730f4ff21aa --- /dev/null +++ b/docs/apache-airflow-providers-teradata/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:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-teradata/operators/index.rst b/docs/apache-airflow-providers-teradata/operators/index.rst new file mode 100644 index 00000000000000..b01345d8fd148c --- /dev/null +++ b/docs/apache-airflow-providers-teradata/operators/index.rst @@ -0,0 +1,25 @@ + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +Teradata operators +=================== + +.. toctree:: + :maxdepth: 1 + :glob: + + * diff --git a/docs/apache-airflow-providers-teradata/operators/teradata.rst b/docs/apache-airflow-providers-teradata/operators/teradata.rst new file mode 100644 index 00000000000000..8b902ad9afcc0c --- /dev/null +++ b/docs/apache-airflow-providers-teradata/operators/teradata.rst @@ -0,0 +1,36 @@ + .. 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/operator:TeradataOperator: + +TeradataOperator +================ + +The purpose of TeradataOperator is to define tasks involving interactions with the Teradata. + +Execute SQL in an Teradata +--------------------------------- + +To execute arbitrary SQL in an Teradata, use the +:class:`~airflow.providers.teradata.operators.teradata.TeradataOperator`. + +An example of executing a simple query is as follows: + +.. exampleinclude:: /../../airflow/providers/teradata/example_dags/example_teradata_operator.py + :language: python + :start-after: [START howto_teradata_operator] + :end-before: [END howto_teradata_operator] diff --git a/docs/apache-airflow-providers-teradata/operators/teradata_to_teradata.rst b/docs/apache-airflow-providers-teradata/operators/teradata_to_teradata.rst new file mode 100644 index 00000000000000..9c010be5a61a1e --- /dev/null +++ b/docs/apache-airflow-providers-teradata/operators/teradata_to_teradata.rst @@ -0,0 +1,38 @@ + .. 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/operator:TeradataToTeradataOperator: + +TeradataToTeradataOperator +========================== + +The purpose of TeradataToTeradataOperator is to define tasks involving data transfer between two Teradata instances. +Use the :class:`TeradataToTeradataOperator ` +to transfer data between two Teradata instances. + +Transfer data between two Teradata instances +----------------------------------------------- + +To transfer data between two Teradata instances, use the +:class:`~airflow.providers.teradata.transfers.teradata_to_teradata.TeradataToTeradataOperator`. + +An example usage of the TeradataToTeradataOperator is as follows: + +.. exampleinclude:: /../../airflow/providers/teradata/example_dags/example_teradata_to_teradata_transfer_operator.py + :language: python + :start-after: [START howto_transfer_operator_teradata_to_teradata] + :end-before: [END howto_transfer_operator_teradata_to_teradata] diff --git a/docs/apache-airflow-providers-teradata/redirects.txt b/docs/apache-airflow-providers-teradata/redirects.txt new file mode 100644 index 00000000000000..74ee3f22fa30c8 --- /dev/null +++ b/docs/apache-airflow-providers-teradata/redirects.txt @@ -0,0 +1 @@ +connections/index.rst connections/teradata.rst diff --git a/docs/apache-airflow-providers-teradata/security.rst b/docs/apache-airflow-providers-teradata/security.rst new file mode 100644 index 00000000000000..66c6f79a4ecfcb --- /dev/null +++ b/docs/apache-airflow-providers-teradata/security.rst @@ -0,0 +1,38 @@ + + .. 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. + +Releasing security patches +-------------------------- + +Airflow providers are released independently from Airflow itself and the information about vulnerabilities +is published separately. You can upgrade providers independently from Airflow itself, following the +instructions found in :doc:`apache-airflow:installation/installing-from-pypi`. + +When we release Provider version, the development is always done from the ``main`` branch where we prepare +the next version. The provider uses strict `SemVer `_ versioning policy. Depending on +the scope of the change, Provider will get ''MAJOR'' version upgrade when there are +breaking changes, ``MINOR`` version upgrade when there are new features or ``PATCHLEVEL`` version upgrade +when there are only bug fixes (including security bugfixes) - and this is the only version that receives +security fixes by default, so you should upgrade to latest version of the provider if you want to receive +all released security fixes. + +The only exception to that rule is when we have a critical security fix and good reason to provide an +out-of-band release for the provider, in which case stakeholders in the provider might decide to cherry-pick +and prepare a branch for an older version of the provider following the +`mixed governance model `_ +and requires interested parties to cherry-pick and test the fixes. diff --git a/docs/apache-airflow/extra-packages-ref.rst b/docs/apache-airflow/extra-packages-ref.rst index 8c1c1143c53184..7549c5ed08be9f 100644 --- a/docs/apache-airflow/extra-packages-ref.rst +++ b/docs/apache-airflow/extra-packages-ref.rst @@ -289,6 +289,8 @@ Some of those enable Airflow to use executors to run tasks with them - other tha +---------------------+-----------------------------------------------------+-----------------------------------------------------------------+----------------------------------------------+ | singularity | ``pip install 'apache-airflow[singularity]'`` | Singularity container operator | | +---------------------+-----------------------------------------------------+-----------------------------------------------------------------+----------------------------------------------+ +| teradata | ``pip install 'apache-airflow[teradata]'`` | Teradata hooks and operators | | ++---------------------+-----------------------------------------------------+-----------------------------------------------------------------+----------------------------------------------+ | trino | ``pip install 'apache-airflow[trino]'`` | All Trino related operators & hooks | | +---------------------+-----------------------------------------------------+-----------------------------------------------------------------+----------------------------------------------+ diff --git a/docs/integration-logos/teradata/Teradata.png b/docs/integration-logos/teradata/Teradata.png new file mode 100644 index 0000000000000000000000000000000000000000..d7afefb4fc64b747f1e664c71be4e35ac8e652df GIT binary patch literal 2220 zcmV;d2vhe`Nk&Gb2mku6)))IZ>=FAh>3REW@L&6X`+4mX|Nj23)?df1H?WCAS_9*55mbXn3ujkIQDSJ%P zZY4BBS^^;7PC#z9pJn!#H&f_78hJYaGERnwxcS`8+!_YlR!gJM-%||P{f?cQmbFXt zuNS^^Sie-2DrR^uW%3&wg$jC&6mZaZHzq{J*hS2CPo|%>cRxbiz$B&H!o2OzbJf`^ zK{^Oqu!WcAvFswm2N-jcfB^pg{)2Wd&PTt&(S^&dYbAe2F%+?Av?oF!^L>JpE0!i2 z-K5y{xybv(!|*nW|KB;ZSt84>4iU5@m21u6-8KG$a`7g-H{<}^Oo=`5o@%qF|Cj)# z+E5pVZ=UblL+F!>C%$dKmCX%Od{pFEEk6Ehhu$yADq1oKx1M(sxBvIa#hcPa^FIVM z9$fixdCz8V_uWs~f0`BLmCeC+<`(gvPv8lh+M)YumT=l%XCf>KDY}1qu%U4C0>K5n zm;NR=wcxl8@^!{G;UCEL^=*&;?eG7o0R`m>rE9)c-2OGuzvIp~dGIv9q7M^vlW>jX z#4A4tX)*KJt~1&108l2%Sru)>kW7Am`29)L+5QeHkbuC->T^&8{^w6~e-p?%W$RdD zku(2B=*E10d*~FfwxZ_q#u$79?r-2h_uoC%cvuTICbxzP(`LnvTG_ zy<-%flKqhO;HK^;zM@v?uN5|nSQ&EQhMX}qYC2ng0hXvqr!Y4(b+G%+CsNG{_&_k; zItW*3R#p#}34d^%0cG-LG3PnKY=tZ1&_CYA&f=vI;z+t?_IzN+lMK`Bz~m_rg`u)F zqlB|hYhCxGd5em}gyjbsagx9?>VG6Y!Q?1#2BO{q%@H^E3~yjA6m1MULkB91%FY)A z;a5xsZxXu96r$9kgC0a>)^L(zxC;w)R(za=-(EWueb?em3^aU10BNU8fXf<@TkWS& zSH(Ga@eff0DoYU#-QUBX?z~&9kXDx^X`A)5UIaZ3jeH_9+`u1(Dn{&9cjvcsbE&ZW zZ)M>vCs>{3~6AML}!HBpi8Pv9O|U3{3`GJD;Q|KIGV@S;@z82k7Z zmx6DIZ<5F_use90eLPMa1#Z#F1CMsdiuyA6-?6-;TV};toN+tijWR+4FLMHz?yv2N zA#C;yXXH&(SiFkp36swbNHt@+aP!I!8NMu%S{?Q+tzyx>r4?+#^D`J!#@~SyZ(S7T z&A>@^WqR~}0^?7YkH>mgtNWkdiSIP|3KxcJO1FFzEx-)#2^llQtpZZrYHU&fY$n51 zCYfeFOuHoEza{qE0WpKZ@Npc-Sq;Wn|7XK|l1!DRrFl)Xbb2qU`>r-Zs2MQ>ZD6RM z^bV-Mg>if*O36Uk^;ks^%3?MYad58MFko4ADX7n(R%2jt*oWPt5nOZPzW(udwvLk; za2sdED6rIF#$$;oMyN3y9(Ijdcp5EE4OCexzyK4GP7NnU5%xHR+7?PoZ$Qc6fi=Ah zEH73i=FuM+%-^g*>f@)>m{*_3f2R{?Eq=qAXgw$=cMWD`pJ^A+@Qb8>p^z#X#b8=1.3.1", + "apache-airflow>=2.6.0", + "teradatasql>=17.20.0.28", + "teradatasqlalchemy>=17.20.0.0" + ], + "devel-deps": [], + "cross-providers-deps": [ + "common.sql" + ], + "excluded-python-versions": [], + "state": "ready" + }, "trino": { "deps": [ "apache-airflow-providers-common-sql>=1.3.1", diff --git a/images/breeze/output_build-docs.svg b/images/breeze/output_build-docs.svg new file mode 100644 index 00000000000000..224955b755c7a2 --- /dev/null +++ b/images/breeze/output_build-docs.svg @@ -0,0 +1,217 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: build-docs + + + + + + + + + + +Usage:                                                                                                                 +breeze build-docs                                                                                                      +[OPTIONS] [airbyte | alibaba | all-providers | amazon | apache-airflow | apache-airflow-providers | apache.beam |      +apache.cassandra | apache.drill | apache.druid | apache.flink | apache.hdfs | apache.hive | apache.impala |            +apache.kafka | apache.kylin | apache.livy | apache.pig | apache.pinot | apache.spark | apprise | arangodb | asana |    +atlassian.jira | celery | cloudant | cncf.kubernetes | cohere | common.io | common.sql | databricks | datadog |        +dbt.cloud | dingding | discord | docker | docker-stack | elasticsearch | exasol | fab | facebook | ftp | github |      +google | grpc | hashicorp | helm-chart | http | imap | influxdb | jdbc | jenkins | microsoft.azure | microsoft.mssql | +microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage | opensearch |       +opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | redis | salesforce | samba |     +segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular | telegram |     +teradata | trino | vertica | weaviate | yandex | zendesk]...                                                           + +Build documents. + +╭─ Doc flags â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â•® +│--docs-only-dOnly build documentation.│ +│--spellcheck-only-sOnly run spell checking.│ +│--clean-buildClean inventories of Inter-Sphinx documentation and generated APIs and sphinx    â”‚ +│artifacts before the build - useful for a clean build.                           â”‚ +│--one-pass-onlyBuilds documentation in one pass only. This is useful for debugging sphinx       â”‚ +│errors.                                                                          â”‚ +│--package-filterList of packages to consider. You can use the full names like                    â”‚ +│apache-airflow-providers-<provider>, the short hand names or the glob pattern    â”‚ +│matching the full package name. The list of short hand names can be found in     â”‚ +│--help output                                                                    â”‚ +│(TEXT)                                                                           â”‚ +│--include-not-ready-providersWhether to include providers that are not yet ready to be released.│ +│--include-removed-providersWhether to include providers that are removed.│ +│--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow]│ +│--builderBuildx builder used to perform `docker buildx build` commands.(TEXT)│ +│[default: autodetect]                                         â”‚ +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common 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_build-docs.txt b/images/breeze/output_build-docs.txt new file mode 100644 index 00000000000000..c7cd5f6767596e --- /dev/null +++ b/images/breeze/output_build-docs.txt @@ -0,0 +1 @@ +7ca1e6419bdabea001875367b7535132 diff --git a/images/breeze/output_release-management_add-back-references.txt b/images/breeze/output_release-management_add-back-references.txt new file mode 100644 index 00000000000000..fe8308b151af55 --- /dev/null +++ b/images/breeze/output_release-management_add-back-references.txt @@ -0,0 +1 @@ +c70bf125c417ca81094c1ff6d5a454d7 diff --git a/images/breeze/output_release-management_generate-issue-content-providers.txt b/images/breeze/output_release-management_generate-issue-content-providers.txt new file mode 100644 index 00000000000000..f09e6910800a2b --- /dev/null +++ b/images/breeze/output_release-management_generate-issue-content-providers.txt @@ -0,0 +1 @@ +726d926fa9a7e9697b8aa6d9574f4987 diff --git a/images/breeze/output_release-management_prepare-provider-documentation.txt b/images/breeze/output_release-management_prepare-provider-documentation.txt new file mode 100644 index 00000000000000..4512c7c1cad4c2 --- /dev/null +++ b/images/breeze/output_release-management_prepare-provider-documentation.txt @@ -0,0 +1 @@ +4e4201a92d41957856654b1bfd0bb8da diff --git a/images/breeze/output_release-management_prepare-provider-packages.txt b/images/breeze/output_release-management_prepare-provider-packages.txt new file mode 100644 index 00000000000000..c533f6e625f2c1 --- /dev/null +++ b/images/breeze/output_release-management_prepare-provider-packages.txt @@ -0,0 +1 @@ +502cb95a33afb3126793d2d06f77ae43 diff --git a/images/breeze/output_release-management_publish-docs.svg b/images/breeze/output_release-management_publish-docs.svg new file mode 100644 index 00000000000000..48a7830866e434 --- /dev/null +++ b/images/breeze/output_release-management_publish-docs.svg @@ -0,0 +1,227 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: release-management publish-docs + + + + + + + + + + +Usage:                                                                                                                 +breeze release-management publish-docs                                                                                 +[OPTIONS] [airbyte | alibaba | all-providers | amazon | apache-airflow | apache-airflow-providers | apache.beam |      +apache.cassandra | apache.drill | apache.druid | apache.flink | apache.hdfs | apache.hive | apache.impala |            +apache.kafka | apache.kylin | apache.livy | apache.pig | apache.pinot | apache.spark | apprise | arangodb | asana |    +atlassian.jira | celery | cloudant | cncf.kubernetes | cohere | common.io | common.sql | databricks | datadog |        +dbt.cloud | dingding | discord | docker | docker-stack | elasticsearch | exasol | fab | facebook | ftp | github |      +google | grpc | hashicorp | helm-chart | http | imap | influxdb | jdbc | jenkins | microsoft.azure | microsoft.mssql | +microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage | opensearch |       +opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | redis | salesforce | samba |     +segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular | telegram |     +teradata | trino | vertica | weaviate | yandex | zendesk]...                                                           + +Command to publish generated documentation to airflow-site + +╭─ Publish Docs â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â•® +│*--airflow-site-directory-aLocal directory path of cloned airflow-site repo.(DIRECTORY)[required]│ +│--include-not-ready-providersWhether to include providers that are not yet ready to be released.│ +│--include-removed-providersWhether to include providers that are removed.│ +│--override-versioned-sOverrides versioned directories.│ +│--package-filterList of packages to consider. You can use the full names like                 â”‚ +│apache-airflow-providers-<provider>, the short hand names or the glob pattern â”‚ +│matching the full package name. The list of short hand names can be found in  â”‚ +│--help output                                                                 â”‚ +│(TEXT)                                                                        â”‚ +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel running â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â”€â•® +│--run-in-parallelRun the operation in parallel on all or selected subset of parameters.│ +│--parallelismMaximum number of processes to use while running the operation in parallel.│ +│(INTEGER RANGE)                                                            â”‚ +│[default: 4; 1<=x<=8]                                                      â”‚ +│--skip-cleanupSkip cleanup of temporary files created during parallel run.│ +│--debug-resourcesWhether to show resource information while running in parallel.│ +│--include-success-outputsWhether to include outputs of successful parallel runs (skipped by default).│ +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common 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_release-management_publish-docs.txt b/images/breeze/output_release-management_publish-docs.txt new file mode 100644 index 00000000000000..18eb0d66807551 --- /dev/null +++ b/images/breeze/output_release-management_publish-docs.txt @@ -0,0 +1 @@ +d808155d2526f30c334cb9d208bed594 diff --git a/pyproject.toml b/pyproject.toml index 43cb43107a768b..430b6e93cc3a5c 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -533,6 +533,7 @@ alibaba = [ "oss2>=2.14.0", ] amazon = [ + "PyAthena>=3.0.10", "apache-airflow[common_sql]", "apache-airflow[http]", "asgiref", @@ -552,7 +553,6 @@ amazon = [ "mypy-boto3-redshift-data>=1.33.0", "mypy-boto3-s3>=1.33.0", "s3fs>=2023.10.0", - "PyAthena>=3.0.10", ] apache-beam = [ "apache-beam>=2.53.0", @@ -930,6 +930,11 @@ tabular = [ telegram = [ "python-telegram-bot>=20.2", ] +teradata = [ + "apache-airflow[common_sql]", + "teradatasql>=17.20.0.28", + "teradatasqlalchemy>=17.20.0.0", +] trino = [ "apache-airflow[common_sql]", "pandas>=1.2.5", @@ -1057,6 +1062,7 @@ all = [ "apache-airflow[tableau]", "apache-airflow[tabular]", "apache-airflow[telegram]", + "apache-airflow[teradata]", "apache-airflow[trino]", "apache-airflow[vertica]", "apache-airflow[weaviate]", @@ -1156,6 +1162,7 @@ devel-all = [ "apache-airflow[tableau]", "apache-airflow[tabular]", "apache-airflow[telegram]", + "apache-airflow[teradata]", "apache-airflow[trino]", "apache-airflow[vertica]", "apache-airflow[weaviate]", diff --git a/tests/providers/teradata/__init__.py b/tests/providers/teradata/__init__.py new file mode 100644 index 00000000000000..13a83393a9124b --- /dev/null +++ b/tests/providers/teradata/__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/teradata/hooks/__init__.py b/tests/providers/teradata/hooks/__init__.py new file mode 100644 index 00000000000000..217e5db9607827 --- /dev/null +++ b/tests/providers/teradata/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/tests/providers/teradata/hooks/test_teradata.py b/tests/providers/teradata/hooks/test_teradata.py new file mode 100644 index 00000000000000..8bb0b24f220a26 --- /dev/null +++ b/tests/providers/teradata/hooks/test_teradata.py @@ -0,0 +1,274 @@ +# +# 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 __future__ import annotations + +import json +from datetime import datetime +from unittest import mock + +import pytest + +from airflow.models import Connection + +try: + from airflow.providers.teradata.hooks.teradata import TeradataHook + + # from airflow.providers.teradata.hooks.teradata import TeradataHook +except ImportError: + pytest.skip( + "Airflow Provider for Teradata not available, unable to import dependency " + "airflow.providers.teradata.hooks.teradata.TeradataHook", + allow_module_level=True, + ) + + +class TestTeradataHook: + def setup_method(self): + self.connection = Connection( + conn_type="teradata", + login="login", + password="password", + host="host", + schema="schema", + ) + self.db_hook = TeradataHook(teradata_conn_id="teradata_conn_id", database="test_db") + self.db_hook.get_connection = mock.Mock() + self.db_hook.get_connection.return_value = self.connection + self.cur = mock.MagicMock(rowcount=0) + self.conn = mock.MagicMock() + self.conn.cursor.return_value = self.cur + conn = self.conn + + class UnitTestTeradataHook(TeradataHook): + conn_name_attr = "teradata_conn_id" + + def get_conn(self): + return conn + + self.test_db_hook = UnitTestTeradataHook() + + @mock.patch("teradatasql.connect") + def test_get_conn(self, mock_connect): + self.db_hook.get_conn() + assert mock_connect.call_count == 1 + args, kwargs = mock_connect.call_args + assert args == () + assert kwargs["host"] == "host" + assert kwargs["database"] == "schema" + assert kwargs["dbs_port"] == "1025" + assert kwargs["user"] == "login" + assert kwargs["password"] == "password" + + @mock.patch("teradatasql.connect") + def test_get_tmode_conn(self, mock_connect): + tmode_name = {"tmode": "tera"} + self.connection.extra = json.dumps(tmode_name) + self.db_hook.get_conn() + assert mock_connect.call_count == 1 + args, kwargs = mock_connect.call_args + assert args == () + assert kwargs["host"] == "host" + assert kwargs["database"] == "schema" + assert kwargs["dbs_port"] == "1025" + assert kwargs["user"] == "login" + assert kwargs["password"] == "password" + assert kwargs["tmode"] == "tera" + + @mock.patch("teradatasql.connect") + def test_get_sslmode_conn(self, mock_connect): + tmode_name = {"sslmode": "require"} + self.connection.extra = json.dumps(tmode_name) + self.db_hook.get_conn() + assert mock_connect.call_count == 1 + args, kwargs = mock_connect.call_args + assert args == () + assert kwargs["host"] == "host" + assert kwargs["database"] == "schema" + assert kwargs["dbs_port"] == "1025" + assert kwargs["user"] == "login" + assert kwargs["password"] == "password" + assert kwargs["sslmode"] == "require" + + @mock.patch("teradatasql.connect") + def test_get_sslverifyca_conn(self, mock_connect): + extravalues = {"sslmode": "verify-ca", "sslca": "/tmp/cert"} + self.connection.extra = json.dumps(extravalues) + self.db_hook.get_conn() + assert mock_connect.call_count == 1 + args, kwargs = mock_connect.call_args + assert args == () + assert kwargs["host"] == "host" + assert kwargs["database"] == "schema" + assert kwargs["dbs_port"] == "1025" + assert kwargs["user"] == "login" + assert kwargs["password"] == "password" + assert kwargs["sslmode"] == "verify-ca" + assert kwargs["sslca"] == "/tmp/cert" + + @mock.patch("teradatasql.connect") + def test_get_sslverifyfull_conn(self, mock_connect): + extravalues = {"sslmode": "verify-full", "sslca": "/tmp/cert"} + self.connection.extra = json.dumps(extravalues) + self.db_hook.get_conn() + assert mock_connect.call_count == 1 + args, kwargs = mock_connect.call_args + assert args == () + assert kwargs["host"] == "host" + assert kwargs["database"] == "schema" + assert kwargs["dbs_port"] == "1025" + assert kwargs["user"] == "login" + assert kwargs["password"] == "password" + assert kwargs["sslmode"] == "verify-full" + assert kwargs["sslca"] == "/tmp/cert" + + @mock.patch("teradatasql.connect") + def test_get_sslcrc_conn(self, mock_connect): + extravalues = {"sslcrc": "sslcrc"} + self.connection.extra = json.dumps(extravalues) + self.db_hook.get_conn() + assert mock_connect.call_count == 1 + args, kwargs = mock_connect.call_args + assert args == () + assert kwargs["host"] == "host" + assert kwargs["database"] == "schema" + assert kwargs["dbs_port"] == "1025" + assert kwargs["user"] == "login" + assert kwargs["password"] == "password" + assert kwargs["sslcrc"] == "sslcrc" + + @mock.patch("teradatasql.connect") + def test_get_sslprotocol_conn(self, mock_connect): + extravalues = {"sslprotocol": "protocol"} + self.connection.extra = json.dumps(extravalues) + self.db_hook.get_conn() + assert mock_connect.call_count == 1 + args, kwargs = mock_connect.call_args + assert args == () + assert kwargs["host"] == "host" + assert kwargs["database"] == "schema" + assert kwargs["dbs_port"] == "1025" + assert kwargs["user"] == "login" + assert kwargs["password"] == "password" + assert kwargs["sslprotocol"] == "protocol" + + @mock.patch("teradatasql.connect") + def test_get_sslcipher_conn(self, mock_connect): + extravalues = {"sslcipher": "cipher"} + self.connection.extra = json.dumps(extravalues) + self.db_hook.get_conn() + assert mock_connect.call_count == 1 + args, kwargs = mock_connect.call_args + assert args == () + assert kwargs["host"] == "host" + assert kwargs["database"] == "schema" + assert kwargs["dbs_port"] == "1025" + assert kwargs["user"] == "login" + assert kwargs["password"] == "password" + assert kwargs["sslcipher"] == "cipher" + + @mock.patch("sqlalchemy.create_engine") + def test_get_sqlalchemy_conn(self, mock_connect): + self.db_hook.get_sqlalchemy_engine() + assert mock_connect.call_count == 1 + args = mock_connect.call_args.args + assert len(args) == 1 + expected_link = ( + f"teradatasql://{self.connection.login}:{self.connection.password}@{self.connection.host}" + ) + assert expected_link == args[0] + + def test_get_uri(self): + ret_uri = self.db_hook.get_uri() + expected_uri = f"teradata://{self.connection.login}:{self.connection.password}@{self.connection.host}/{self.connection.schema}" + assert expected_uri == ret_uri + + def test_get_records(self): + sql = "SQL" + self.test_db_hook.get_records(sql) + self.cur.execute.assert_called_once_with(sql) + assert self.conn.commit.called + + def test_run_without_parameters(self): + sql = "SQL" + self.test_db_hook.run(sql) + self.cur.execute.assert_called_once_with(sql) + assert self.conn.commit.called + + def test_run_with_parameters(self): + sql = "SQL" + param = ("p1", "p2") + self.test_db_hook.run(sql, parameters=param) + self.cur.execute.assert_called_once_with(sql, param) + assert self.conn.commit.called + + def test_insert_rows(self): + rows = [ + ( + "'test_string", + None, + datetime(2023, 8, 15), + 1, + 3.14, + "str", + ) + ] + target_fields = [ + "basestring", + "none", + "datetime", + "int", + "float", + "str", + ] + self.test_db_hook.insert_rows("table", rows, target_fields) + self.cur.execute.assert_called_once_with( + "INSERT INTO table (basestring, none, datetime, int, float, str) VALUES (?,?,?,?,?,?)", + ("'test_string", None, "2023-08-15T00:00:00", "1", "3.14", "str"), + ) + + def test_bulk_insert_rows_with_fields(self): + rows = [(1, 2, 3), (4, 5, 6), (7, 8, 9)] + target_fields = ["col1", "col2", "col3"] + self.test_db_hook.bulk_insert_rows("table", rows, target_fields) + self.cur.executemany.assert_called_once_with( + "INSERT INTO table (col1, col2, col3) VALUES (?, ?, ?)", rows + ) + + def test_bulk_insert_rows_with_commit_every(self): + rows = [(1, 2, 3), (4, 5, 6), (7, 8, 9)] + target_fields = ["col1", "col2", "col3"] + self.test_db_hook.bulk_insert_rows("table", rows, target_fields, commit_every=2) + calls = [ + mock.call("INSERT INTO table (col1, col2, col3) values (1, 2, 3)"), + mock.call("INSERT INTO table (col1, col2, col3) values (1, 2, 3)"), + ] + calls = [ + mock.call("INSERT INTO table (col1, col2, col3) VALUES (?, ?, ?)", rows[:2]), + mock.call("INSERT INTO table (col1, col2, col3) VALUES (?, ?, ?)", rows[2:]), + ] + self.cur.executemany.assert_has_calls(calls, any_order=True) + + def test_bulk_insert_rows_without_fields(self): + rows = [(1, 2, 3), (4, 5, 6), (7, 8, 9)] + self.test_db_hook.bulk_insert_rows("table", rows) + self.cur.executemany.assert_called_once_with("INSERT INTO table VALUES (?, ?, ?)", rows) + + def test_bulk_insert_rows_no_rows(self): + rows = [] + with pytest.raises(ValueError): + self.test_db_hook.bulk_insert_rows("table", rows) diff --git a/tests/providers/teradata/operators/__init__.py b/tests/providers/teradata/operators/__init__.py new file mode 100644 index 00000000000000..217e5db9607827 --- /dev/null +++ b/tests/providers/teradata/operators/__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/tests/providers/teradata/operators/test_teradata.py b/tests/providers/teradata/operators/test_teradata.py new file mode 100644 index 00000000000000..4dcaf7a924076f --- /dev/null +++ b/tests/providers/teradata/operators/test_teradata.py @@ -0,0 +1,128 @@ +# +# 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 __future__ import annotations + +from unittest import mock +from unittest.mock import MagicMock, Mock + +import pytest + +from airflow.models.dag import DAG +from airflow.providers.common.sql.hooks.sql import fetch_all_handler +from airflow.utils import timezone + +try: + from airflow.providers.teradata.hooks.teradata import TeradataHook + from airflow.providers.teradata.operators.teradata import TeradataOperator +except ImportError: + pytest.skip("Teradata not available", allow_module_level=True) + +from airflow.exceptions import AirflowException + +DEFAULT_DATE = timezone.datetime(2015, 1, 1) +TEST_DAG_ID = "unit_test_dag" + + +class TestTeradataOperator: + def setup_method(self): + args = {"owner": "airflow", "start_date": DEFAULT_DATE} + dag = DAG(TEST_DAG_ID, default_args=args) + self.dag = dag + + @mock.patch("airflow.providers.common.sql.operators.sql.SQLExecuteQueryOperator.get_db_hook") + def test_get_hook_from_conn(self, mock_get_db_hook): + """ + :class:`~.MsSqlOperator` should use the hook returned by :meth:`airflow.models.Connection.get_hook` + if one is returned. + + This behavior is necessary in order to support usage of :class:`~.OdbcHook` with this operator. + + Specifically we verify here that :meth:`~.MsSqlOperator.get_hook` returns the hook returned from a + call of ``get_hook`` on the object returned from :meth:`~.BaseHook.get_connection`. + """ + mock_hook = MagicMock() + mock_get_db_hook.return_value = mock_hook + + op = TeradataOperator(task_id="test", sql="") + assert op.get_db_hook() == mock_hook + + @mock.patch( + "airflow.providers.common.sql.operators.sql.SQLExecuteQueryOperator.get_db_hook", + autospec=TeradataHook, + ) + def test_get_hook_default(self, mock_get_db_hook): + """ + If :meth:`airflow.models.Connection.get_hook` does not return a hook (e.g. because of an invalid + conn type), then :class:`~.TeradataHook` should be used. + """ + mock_get_db_hook.return_value.side_effect = Mock(side_effect=AirflowException()) + + op = TeradataOperator(task_id="test", sql="") + assert op.get_db_hook().__class__.__name__ == "TeradataHook" + + @mock.patch("airflow.providers.common.sql.operators.sql.SQLExecuteQueryOperator.get_db_hook") + def test_execute(self, mock_get_db_hook): + sql = "SELECT * FROM test_table" + teradata_conn_id = "teradata_default" + parameters = {"parameter": "value"} + autocommit = False + context = "test_context" + task_id = "test_task_id" + + operator = TeradataOperator( + sql=sql, + conn_id=teradata_conn_id, + parameters=parameters, + task_id=task_id, + ) + operator.execute(context=context) + mock_get_db_hook.return_value.run.assert_called_once_with( + sql=sql, + autocommit=autocommit, + parameters=parameters, + handler=fetch_all_handler, + return_last=True, + ) + + @mock.patch("airflow.providers.common.sql.operators.sql.SQLExecuteQueryOperator.get_db_hook") + def test_teradata_operator_test_multi(self, mock_get_db_hook): + sql = [ + "CREATE TABLE IF NOT EXISTS test_airflow (dummy VARCHAR(50))", + "TRUNCATE TABLE test_airflow", + "INSERT INTO test_airflow VALUES ('X')", + ] + teradata_conn_id = "teradata_default" + parameters = {"parameter": "value"} + autocommit = False + context = "test_context" + task_id = "test_task_id" + + operator = TeradataOperator( + sql=sql, + conn_id=teradata_conn_id, + parameters=parameters, + task_id=task_id, + ) + operator.execute(context=context) + mock_get_db_hook.return_value.run.assert_called_once_with( + sql=sql, + autocommit=autocommit, + parameters=parameters, + handler=fetch_all_handler, + return_last=True, + ) diff --git a/tests/providers/teradata/transfers/__init__.py b/tests/providers/teradata/transfers/__init__.py new file mode 100644 index 00000000000000..217e5db9607827 --- /dev/null +++ b/tests/providers/teradata/transfers/__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/tests/providers/teradata/transfers/test_teradata_to_teradata.py b/tests/providers/teradata/transfers/test_teradata_to_teradata.py new file mode 100644 index 00000000000000..fc2b45561fac1d --- /dev/null +++ b/tests/providers/teradata/transfers/test_teradata_to_teradata.py @@ -0,0 +1,71 @@ +# +# 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 __future__ import annotations + +from decimal import Decimal +from unittest import mock +from unittest.mock import MagicMock + +from airflow.providers.teradata.transfers.teradata_to_teradata import TeradataToTeradataOperator + + +class TestTeradataToTeradataTransfer: + def test_execute(self): + teradata_destination_conn_id = "teradata_destination_conn_id" + destination_table = "destination_table" + teradata_source_conn_id = "teradata_source_conn_id" + source_sql = (r"""select DATE where DATE > {{ source_sql_params.ref_date }};""",) + source_sql_params = {"ref_date": "2018-01-01"} + rows_chunk = 5000 + cursor_description = [ + ["user_id", Decimal, None, 8, 10, 0, False], + ["user_name", str, None, 60, None, None, True], + ] + + cursor_rows = [[Decimal("1"), "User1"], [Decimal("2"), "User2"], [Decimal("3"), "User3"]] + + mock_dest_hook = MagicMock() + mock_src_hook = MagicMock() + mock_src_conn = mock_src_hook.get_conn.return_value.__enter__.return_value + mock_cursor = mock_src_conn.cursor.return_value + mock_cursor.description.__iter__.return_value = cursor_description + mock_cursor.fetchmany.side_effect = [cursor_rows, []] + + td_transfer_op = TeradataToTeradataOperator( + task_id="transfer_data", + teradata_destination_conn_id=teradata_destination_conn_id, + destination_table=destination_table, + teradata_source_conn_id=teradata_source_conn_id, + source_sql=source_sql, + source_sql_params=source_sql_params, + rows_chunk=rows_chunk, + ) + + td_transfer_op._execute(mock_src_hook, mock_dest_hook, None) + + assert mock_src_hook.get_conn.called + assert mock_src_conn.cursor.called + mock_cursor.execute.assert_called_once_with(source_sql, source_sql_params) + + calls = [ + mock.call(rows_chunk), + ] + mock_cursor.fetchmany.assert_has_calls(calls) + mock_dest_hook.bulk_insert_rows.assert_called_once_with( + destination_table, cursor_rows, commit_every=rows_chunk, target_fields=["user_id", "user_name"] + ) diff --git a/tests/system/providers/teradata/__init__.py b/tests/system/providers/teradata/__init__.py new file mode 100644 index 00000000000000..13a83393a9124b --- /dev/null +++ b/tests/system/providers/teradata/__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/system/providers/teradata/create_ssl_table.sql b/tests/system/providers/teradata/create_ssl_table.sql new file mode 100644 index 00000000000000..f82b38ab4ea17c --- /dev/null +++ b/tests/system/providers/teradata/create_ssl_table.sql @@ -0,0 +1,22 @@ +/* + 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. +*/ +CREATE TABLE SSL_Users, FALLBACK ( + username varchar(50), + description varchar(256) + ); diff --git a/tests/system/providers/teradata/create_table.sql b/tests/system/providers/teradata/create_table.sql new file mode 100644 index 00000000000000..ac169deae9602f --- /dev/null +++ b/tests/system/providers/teradata/create_table.sql @@ -0,0 +1,22 @@ +/* + 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. +*/ +CREATE TABLE Users, FALLBACK ( + username varchar(50), + description varchar(256) + ); diff --git a/tests/system/providers/teradata/example_ssl_teradata.py b/tests/system/providers/teradata/example_ssl_teradata.py new file mode 100644 index 00000000000000..f2a69fd5b13b2e --- /dev/null +++ b/tests/system/providers/teradata/example_ssl_teradata.py @@ -0,0 +1,127 @@ +# +# 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. +""" +Example use of Teradata related operators. +""" +from __future__ import annotations + +import datetime +import os + +import pytest + +from airflow import DAG + +try: + from airflow.providers.teradata.operators.teradata import TeradataOperator +except ImportError: + pytest.skip("TERADATA provider not available", allow_module_level=True) + +# [START teradata_operator_howto_guide] + + +# create_table_teradata, insert_teradata_task, create_table_teradata_from_external_file, +# populate_user_table get_all_countries, get_all_description, get_countries_from_continent, +# drop_table_teradata_task, drop_users_table_teradata_task are examples of tasks created by instantiating +# the Teradata Operator + +ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID") +DAG_ID = "example_teradata" + +with DAG( + dag_id=DAG_ID, + start_date=datetime.datetime(2020, 2, 2), + schedule="@once", + catchup=False, + default_args={"conn_id": "teradata_ssl_default"}, +) as dag: + # [START teradata_operator_howto_guide_create_country_table] + create_country_table = TeradataOperator( + task_id="create_country_table", + sql=r""" + CREATE TABLE SSL_Country ( + country_id INTEGER, + name CHAR(25), + continent CHAR(25) + ); + """, + trigger_rule="all_done", + ) + # [END teradata_operator_howto_guide_create_country_table] + # [START teradata_operator_howto_guide_populate_country_table] + populate_country_table = TeradataOperator( + task_id="populate_country_table", + sql=r""" + INSERT INTO SSL_Country VALUES ( 1, 'India', 'Asia'); + INSERT INTO SSL_Country VALUES ( 2, 'Germany', 'Europe'); + INSERT INTO SSL_Country VALUES ( 3, 'Argentina', 'South America'); + INSERT INTO SSL_Country VALUES ( 4, 'Ghana', 'Africa'); + """, + ) + # [END teradata_operator_howto_guide_populate_country_table] + # [START teradata_operator_howto_guide_create_users_table_from_external_file] + create_users_table_from_external_file = TeradataOperator( + task_id="create_users_table_from_external_file", + sql="create_ssl_table.sql", + dag=dag, + ) + # [END teradata_operator_howto_guide_create_users_table_from_external_file] + # [START teradata_operator_howto_guide_get_all_countries] + get_all_countries = TeradataOperator( + task_id="get_all_countries", + sql=r"""SELECT * FROM SSL_Country;""", + ) + # [END teradata_operator_howto_guide_get_all_countries] + # [START teradata_operator_howto_guide_params_passing_get_query] + get_countries_from_continent = TeradataOperator( + task_id="get_countries_from_continent", + sql=r"""SELECT * FROM SSL_Country where {{ params.column }}='{{ params.value }}';""", + params={"column": "continent", "value": "Asia"}, + ) + # [END teradata_operator_howto_guide_params_passing_get_query] + # [START teradata_operator_howto_guide_drop_country_table] + drop_country_table = TeradataOperator( + task_id="drop_country_table", sql=r"""DROP TABLE SSL_Country;""", dag=dag + ) + # [END teradata_operator_howto_guide_drop_country_table] + # [START teradata_operator_howto_guide_drop_users_table] + drop_users_table = TeradataOperator(task_id="drop_users_table", sql=r"""DROP TABLE SSL_Users;""", dag=dag) + # [END teradata_operator_howto_guide_drop_users_table] + + ( + create_country_table + >> populate_country_table + >> create_users_table_from_external_file + >> get_all_countries + >> get_countries_from_continent + >> drop_country_table + >> drop_users_table + ) + + # [END teradata_operator_howto_guide] + + from tests.system.utils.watcher import watcher + + # This test needs watcher in order to properly mark success/failure + # when "tearDown" task with trigger rule is part of the DAG + list(dag.tasks) >> watcher() + +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) diff --git a/tests/system/providers/teradata/example_teradata.py b/tests/system/providers/teradata/example_teradata.py new file mode 100644 index 00000000000000..a7fd039c1d95b7 --- /dev/null +++ b/tests/system/providers/teradata/example_teradata.py @@ -0,0 +1,125 @@ +# +# 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. +""" +Example use of Teradata related operators. +""" +from __future__ import annotations + +import datetime +import os + +import pytest + +from airflow import DAG + +try: + from airflow.providers.teradata.operators.teradata import TeradataOperator +except ImportError: + pytest.skip("TERADATA provider not available", allow_module_level=True) + +# [START teradata_operator_howto_guide] + + +# create_table_teradata, insert_teradata_task, create_table_teradata_from_external_file, populate_user_table +# get_all_countries, get_all_description, get_countries_from_continent, drop_table_teradata_task, drop_users_table_teradata_task +# are examples of tasks created by instantiating the Teradata Operator + +ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID") +DAG_ID = "example_teradata" + +with DAG( + dag_id=DAG_ID, + start_date=datetime.datetime(2020, 2, 2), + schedule="@once", + catchup=False, + default_args={"conn_id": "teradata_default"}, +) as dag: + # [START teradata_operator_howto_guide_create_country_table] + create_country_table = TeradataOperator( + task_id="create_country_table", + sql=r""" + CREATE TABLE Country ( + country_id INTEGER, + name CHAR(25), + continent CHAR(25) + ); + """, + ) + # [END teradata_operator_howto_guide_create_country_table] + # [START teradata_operator_howto_guide_populate_country_table] + populate_country_table = TeradataOperator( + task_id="populate_country_table", + sql=r""" + INSERT INTO Country VALUES ( 1, 'India', 'Asia'); + INSERT INTO Country VALUES ( 2, 'Germany', 'Europe'); + INSERT INTO Country VALUES ( 3, 'Argentina', 'South America'); + INSERT INTO Country VALUES ( 4, 'Ghana', 'Africa'); + """, + ) + # [END teradata_operator_howto_guide_populate_country_table] + # [START teradata_operator_howto_guide_create_users_table_from_external_file] + create_users_table_from_external_file = TeradataOperator( + task_id="create_users_table_from_external_file", + sql="create_table.sql", + dag=dag, + ) + # [END teradata_operator_howto_guide_create_users_table_from_external_file] + # [START teradata_operator_howto_guide_get_all_countries] + get_all_countries = TeradataOperator( + task_id="get_all_countries", + sql=r"""SELECT * FROM Country;""", + ) + # [END teradata_operator_howto_guide_get_all_countries] + # [START teradata_operator_howto_guide_params_passing_get_query] + get_countries_from_continent = TeradataOperator( + task_id="get_countries_from_continent", + sql=r"""SELECT * FROM Country where {{ params.column }}='{{ params.value }}';""", + params={"column": "continent", "value": "Asia"}, + ) + # [END teradata_operator_howto_guide_params_passing_get_query] + # [START teradata_operator_howto_guide_drop_country_table] + drop_country_table = TeradataOperator( + task_id="drop_country_table", sql=r"""DROP TABLE Country;""", dag=dag + ) + # [END teradata_operator_howto_guide_drop_country_table] + # [START teradata_operator_howto_guide_drop_users_table] + drop_users_table = TeradataOperator(task_id="drop_users_table", sql=r"""DROP TABLE Users;""", dag=dag) + # [END teradata_operator_howto_guide_drop_users_table] + + ( + create_country_table + >> populate_country_table + >> create_users_table_from_external_file + >> get_all_countries + >> get_countries_from_continent + >> drop_country_table + >> drop_users_table + ) + + # [END teradata_operator_howto_guide] + + from tests.system.utils.watcher import watcher + + # This test needs watcher in order to properly mark success/failure + # when "tearDown" task with trigger rule is part of the DAG + list(dag.tasks) >> watcher() + +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) diff --git a/tests/system/providers/teradata/example_teradata_to_teradata_transfer.py b/tests/system/providers/teradata/example_teradata_to_teradata_transfer.py new file mode 100644 index 00000000000000..aac9e151a9c15b --- /dev/null +++ b/tests/system/providers/teradata/example_teradata_to_teradata_transfer.py @@ -0,0 +1,179 @@ +# +# 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. +""" +Example Airflow DAG to show usage of teradata to teradata transfer operator + +The transfer operator connects to source teradata server, runs query to fetch data from source +and inserts that data into destination teradata database server. It assumes tables already exists. +The example DAG below assumes Airflow Connection with connection id `teradata_default` already exists. +It creates sample my_users table at source and destination, sets up sample data at source and then +runs transfer operator to copy data to corresponding table on destination server. +""" +from __future__ import annotations + +import datetime +import os + +import pytest + +from airflow import DAG + +try: + from airflow.providers.teradata.operators.teradata import TeradataOperator + from airflow.providers.teradata.transfers.teradata_to_teradata import TeradataToTeradataOperator +except ImportError: + pytest.skip("Teradata provider apache-airflow-provider-teradata not available", allow_module_level=True) + +# [START teradata_to_teradata_transfer_operator_howto_guide] + + +# create_src_table, create_dest_table, insert_data_src, read_data_src, read_data_dest, drop_src_table +# and drop_dest_table are examples of tasks created by instantiating the Teradata Operator and transfer_data +# is the example of task created by instantiating the TeradataToTeradata Transfer Operator. + +ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID") +DAG_ID = "example_teradata_to_teradata_transfer_operator" +CONN_ID = "teradata_default" + +with DAG( + dag_id=DAG_ID, + start_date=datetime.datetime(2020, 2, 2), + schedule="@once", + catchup=False, + default_args={"conn_id": "teradata_default"}, +) as dag: + # [START teradata_to_teradata_transfer_operator_howto_guide_create_src_table] + create_src_table = TeradataOperator( + task_id="create_src_table", + conn_id=CONN_ID, + sql=""" + CREATE TABLE my_users_src, + FALLBACK ( + user_id decimal(10,0) NOT NULL GENERATED ALWAYS AS IDENTITY ( + START WITH 1 + INCREMENT BY 1 + MINVALUE 1 + MAXVALUE 2147483647 + NO CYCLE), + user_name VARCHAR(30), + gender CHAR(1) DEFAULT 'M', + birth_date DATE FORMAT 'YYYY-MM-DD' NOT NULL DEFAULT DATE '2023-01-01' + ) PRIMARY INDEX (user_id); + """, + ) + # [END teradata_to_teradata_transfer_operator_howto_guide_create_src_table] + # [START teradata_to_teradata_transfer_operator_howto_guide_create_dest_table] + create_dest_table = TeradataOperator( + task_id="create_dest_table", + conn_id=CONN_ID, + sql=""" + CREATE TABLE my_users_dest, + FALLBACK ( + user_id decimal(10,0) NOT NULL GENERATED ALWAYS AS IDENTITY ( + START WITH 1 + INCREMENT BY 1 + MINVALUE 1 + MAXVALUE 2147483647 + NO CYCLE), + user_name VARCHAR(30), + gender CHAR(1) DEFAULT 'M', + birth_date DATE FORMAT 'YYYY-MM-DD' NOT NULL DEFAULT DATE '2023-01-01' + ) PRIMARY INDEX (user_id); + """, + ) + # [END teradata_to_teradata_transfer_operator_howto_guide_create_dest_table] + # [START teradata_to_teradata_transfer_operator_howto_guide_insert_data_src] + insert_data_src = TeradataOperator( + task_id="insert_data_src", + conn_id=CONN_ID, + sql=""" + INSERT INTO my_users_src(user_name) VALUES ('User1'); + INSERT INTO my_users_src(user_name) VALUES ('User2'); + INSERT INTO my_users_src(user_name) VALUES ('User3'); + """, + ) + # [END teradata_to_teradata_transfer_operator_howto_guide_insert_data_src] + # [START teradata_to_teradata_transfer_operator_howto_guide_read_data_src] + read_data_src = TeradataOperator( + task_id="read_data_src", + conn_id=CONN_ID, + sql=""" + SELECT TOP 10 * from my_users_src order by user_id desc; + """, + ) + # [END teradata_to_teradata_transfer_operator_howto_guide_read_data_src] + # [START teradata_to_teradata_transfer_operator_howto_guide_transfer_data] + transfer_data = TeradataToTeradataOperator( + task_id="transfer_data", + teradata_destination_conn_id="teradata_default", + destination_table="my_users_dest", + teradata_source_conn_id="teradata_default", + source_sql="select * from my_users_src", + source_sql_params={}, + rows_chunk=2, + ) + # [END teradata_to_teradata_transfer_operator_howto_guide_transfer_data] + # [START teradata_to_teradata_transfer_operator_howto_guide_read_data_dest] + read_data_dest = TeradataOperator( + task_id="read_data_dest", + conn_id=CONN_ID, + sql=""" + SELECT TOP 10 * from my_users_dest order by user_id desc; + """, + ) + # [END teradata_to_teradata_transfer_operator_howto_guide_read_data_dest] + # [START teradata_to_teradata_transfer_operator_howto_guide_drop_src_table] + drop_src_table = TeradataOperator( + task_id="drop_src_table", + conn_id=CONN_ID, + sql=""" + DROP TABLE my_users_src; + """, + ) + # [END teradata_to_teradata_transfer_operator_howto_guide_drop_src_table] + # [START teradata_to_teradata_transfer_operator_howto_guide_drop_dest_table] + drop_dest_table = TeradataOperator( + task_id="drop_dest_table", + conn_id=CONN_ID, + sql=""" + DROP TABLE my_users_dest; + """, + ) + # [END teradata_to_teradata_transfer_operator_howto_guide_drop_dest_table] + ( + create_src_table + >> create_dest_table + >> insert_data_src + >> read_data_src + >> transfer_data + >> read_data_dest + >> drop_src_table + >> drop_dest_table + ) + # [END teradata_to_teradata_transfer_operator_howto_guide] + + from tests.system.utils.watcher import watcher + + # This test needs watcher in order to properly mark success/failure + # when "tearDown" task with trigger rule is part of the DAG + list(dag.tasks) >> watcher() + +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 dbb14e3e7c82c1e30182ccaef045ba28ede2cfee Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Mon, 22 Jan 2024 07:43:58 -0800 Subject: [PATCH 02/21] Static check issues fixed --- CONTRIBUTING.rst | 3 ++- dev/breeze/doc/images/output_build-docs.svg | 2 +- dev/breeze/doc/images/output_build-docs.txt | 2 +- .../images/output_release-management_add-back-references.txt | 2 +- ...ut_release-management_generate-issue-content-providers.txt | 2 +- ...tput_release-management_prepare-provider-documentation.txt | 2 +- .../output_release-management_prepare-provider-packages.txt | 2 +- .../doc/images/output_release-management_publish-docs.svg | 2 +- .../doc/images/output_release-management_publish-docs.txt | 2 +- images/breeze/output_build-docs.svg | 4 ++-- images/breeze/output_release-management_publish-docs.svg | 4 ++-- 11 files changed, 14 insertions(+), 13 deletions(-) diff --git a/CONTRIBUTING.rst b/CONTRIBUTING.rst index 9be33bae34ddb5..457386b4c9b6bd 100644 --- a/CONTRIBUTING.rst +++ b/CONTRIBUTING.rst @@ -900,7 +900,8 @@ microsoft-mssql, microsoft-psrp, microsoft-winrm, mongo, mssql, mysql, neo4j, od openfaas, openlineage, opensearch, opsgenie, oracle, otel, pagerduty, pandas, papermill, password, pgvector, pinecone, pinot, postgres, presto, rabbitmq, redis, s3, s3fs, salesforce, samba, saml, segment, sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd, -tableau, tabular, telegram, teradata, trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex, zendesk +tableau, tabular, telegram, teradata, trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex, +zendesk .. END REGULAR EXTRAS HERE Provider packages diff --git a/dev/breeze/doc/images/output_build-docs.svg b/dev/breeze/doc/images/output_build-docs.svg index dabcb6685415b7..224955b755c7a2 100644 --- a/dev/breeze/doc/images/output_build-docs.svg +++ b/dev/breeze/doc/images/output_build-docs.svg @@ -184,7 +184,7 @@ microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage | opensearch |       opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | redis | salesforce | samba |     segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular | telegram |     -trino | vertica | weaviate | yandex | zendesk]...                                                                      +teradata | trino | vertica | weaviate | yandex | zendesk]...                                                           Build documents. diff --git a/dev/breeze/doc/images/output_build-docs.txt b/dev/breeze/doc/images/output_build-docs.txt index b580d9d01c1ff8..c7cd5f6767596e 100644 --- a/dev/breeze/doc/images/output_build-docs.txt +++ b/dev/breeze/doc/images/output_build-docs.txt @@ -1 +1 @@ -b4bc09e22159b362651e9dd299f795c2 +7ca1e6419bdabea001875367b7535132 diff --git a/dev/breeze/doc/images/output_release-management_add-back-references.txt b/dev/breeze/doc/images/output_release-management_add-back-references.txt index c7d57a09d37710..fe8308b151af55 100644 --- a/dev/breeze/doc/images/output_release-management_add-back-references.txt +++ b/dev/breeze/doc/images/output_release-management_add-back-references.txt @@ -1 +1 @@ -6ceda71ff8edfe80c678c2a6c844f22d +c70bf125c417ca81094c1ff6d5a454d7 diff --git a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt index aa42535eacb958..f09e6910800a2b 100644 --- a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt +++ b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt @@ -1 +1 @@ -8109ff3d327a5e6921b15402f9c15bb6 +726d926fa9a7e9697b8aa6d9574f4987 diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt index 6708878a1863c9..4512c7c1cad4c2 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt @@ -1 +1 @@ -f4dbf1109bcdcca01230e5eb5331fa26 +4e4201a92d41957856654b1bfd0bb8da diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-packages.txt b/dev/breeze/doc/images/output_release-management_prepare-provider-packages.txt index 50a99b308f9b70..c533f6e625f2c1 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-packages.txt +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-packages.txt @@ -1 +1 @@ -e82f390815da62d3927de3f6cb9704f1 +502cb95a33afb3126793d2d06f77ae43 diff --git a/dev/breeze/doc/images/output_release-management_publish-docs.svg b/dev/breeze/doc/images/output_release-management_publish-docs.svg index 1448a8a59448cb..48a7830866e434 100644 --- a/dev/breeze/doc/images/output_release-management_publish-docs.svg +++ b/dev/breeze/doc/images/output_release-management_publish-docs.svg @@ -192,7 +192,7 @@ microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage | opensearch |       opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | redis | salesforce | samba |     segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular | telegram |     -trino | vertica | weaviate | yandex | zendesk]...                                                                      +teradata | trino | vertica | weaviate | yandex | zendesk]...                                                           Command to publish generated documentation to airflow-site diff --git a/dev/breeze/doc/images/output_release-management_publish-docs.txt b/dev/breeze/doc/images/output_release-management_publish-docs.txt index f07d4889c61abb..18eb0d66807551 100644 --- a/dev/breeze/doc/images/output_release-management_publish-docs.txt +++ b/dev/breeze/doc/images/output_release-management_publish-docs.txt @@ -1 +1 @@ -babcac730a3ede766b87ba14ab3484e1 +d808155d2526f30c334cb9d208bed594 diff --git a/images/breeze/output_build-docs.svg b/images/breeze/output_build-docs.svg index 224955b755c7a2..f6003ea42810a7 100644 --- a/images/breeze/output_build-docs.svg +++ b/images/breeze/output_build-docs.svg @@ -168,9 +168,9 @@ - + - + Usage:                                                                                                                 diff --git a/images/breeze/output_release-management_publish-docs.svg b/images/breeze/output_release-management_publish-docs.svg index 48a7830866e434..a11a70c8a8ec55 100644 --- a/images/breeze/output_release-management_publish-docs.svg +++ b/images/breeze/output_release-management_publish-docs.svg @@ -176,9 +176,9 @@ - + - + Usage:                                                                                                                 From 98f0d735024fd67b406738f8023e5f7bf710cfbc Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Tue, 23 Jan 2024 10:12:45 +0530 Subject: [PATCH 03/21] Removed code owner and commented line --- .github/CODEOWNERS | 1 - tests/providers/teradata/hooks/test_teradata.py | 2 -- 2 files changed, 3 deletions(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 230f385662c77f..e58dee9e85fa44 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -75,7 +75,6 @@ airflow/providers/cncf/kubernetes/executors/ @dstandish @jedcunningham /tests/system/providers/amazon/ @eladkal @o-nikolas /tests/providers/common/sql/ @eladkal /tests/providers/slack/ @eladkal -/tests/providers/teradata/ @SatishChGit # Dev tools /.github/workflows/ @potiuk @ashb @kaxil diff --git a/tests/providers/teradata/hooks/test_teradata.py b/tests/providers/teradata/hooks/test_teradata.py index 8bb0b24f220a26..7d0576f0698d5c 100644 --- a/tests/providers/teradata/hooks/test_teradata.py +++ b/tests/providers/teradata/hooks/test_teradata.py @@ -27,8 +27,6 @@ try: from airflow.providers.teradata.hooks.teradata import TeradataHook - - # from airflow.providers.teradata.hooks.teradata import TeradataHook except ImportError: pytest.skip( "Airflow Provider for Teradata not available, unable to import dependency " From e8c4353078aee7c218cb50d1ee46db52353d8e65 Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Wed, 24 Jan 2024 20:09:38 +0530 Subject: [PATCH 04/21] Updated names as per review comments --- .../transfers/teradata_to_teradata.py | 41 ++++++++++--------- 1 file changed, 21 insertions(+), 20 deletions(-) diff --git a/airflow/providers/teradata/transfers/teradata_to_teradata.py b/airflow/providers/teradata/transfers/teradata_to_teradata.py index 2ff491482895a5..9889bee7e1b77e 100644 --- a/airflow/providers/teradata/transfers/teradata_to_teradata.py +++ b/airflow/providers/teradata/transfers/teradata_to_teradata.py @@ -34,47 +34,47 @@ class TeradataToTeradataOperator(BaseOperator): For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:TeradataToTeradataOperator` - :param teradata_destination_conn_id: destination Teradata connection. + :param dest_teradata_conn_id: destination Teradata connection. :param destination_table: destination table to insert rows. - :param teradata_source_conn_id: :ref:`Source Teradata connection `. - :param source_sql: SQL query to execute against the source Teradata database - :param source_sql_params: Parameters to use in sql query. + :param source_teradata_conn_id: :ref:`Source Teradata connection `. + :param sql: SQL query to execute against the source Teradata database + :param sql_params: Parameters to use in sql query. :param rows_chunk: number of rows per chunk to commit. """ template_fields: Sequence[str] = ( - "source_sql", - "source_sql_params", + "sql", + "sql_params", ) template_ext: Sequence[str] = (".sql",) - template_fields_renderers = {"source_sql": "sql", "source_sql_params": "py"} + template_fields_renderers = {"sql": "sql", "sql_params": "py"} ui_color = "#e07c24" def __init__( self, *, - teradata_destination_conn_id: str, + dest_teradata_conn_id: str, destination_table: str, - teradata_source_conn_id: str, - source_sql: str, - source_sql_params: dict | None = None, + source_teradata_conn_id: str, + sql: str, + sql_params: dict | None = None, rows_chunk: int = 5000, **kwargs, ) -> None: super().__init__(**kwargs) - if source_sql_params is None: - source_sql_params = {} - self.teradata_destination_conn_id = teradata_destination_conn_id + if sql_params is None: + sql_params = {} + self.dest_teradata_conn_id = dest_teradata_conn_id self.destination_table = destination_table - self.teradata_source_conn_id = teradata_source_conn_id - self.source_sql = source_sql - self.source_sql_params = source_sql_params + self.source_teradata_conn_id = source_teradata_conn_id + self.sql = sql + self.sql_params = sql_params self.rows_chunk = rows_chunk def _execute(self, src_hook, dest_hook, context) -> None: with src_hook.get_conn() as src_conn: cursor = src_conn.cursor() - cursor.execute(self.source_sql, self.source_sql_params) + cursor.execute(self.sql, self.sql_params) target_fields = [field[0] for field in cursor.description] rows_total = 0 for rows in iter(lambda: cursor.fetchmany(self.rows_chunk), []): @@ -85,7 +85,8 @@ def _execute(self, src_hook, dest_hook, context) -> None: self.log.info("Finished data transfer.") cursor.close() + def execute(self, context: Context) -> None: - src_hook = TeradataHook(teradata_conn_id=self.teradata_source_conn_id) - dest_hook = TeradataHook(teradata_conn_id=self.teradata_destination_conn_id) + src_hook = TeradataHook(teradata_conn_id=self.source_teradata_conn_id) + dest_hook = TeradataHook(teradata_conn_id=self.dest_teradata_conn_id) self._execute(src_hook, dest_hook, context) From 7accb6ab80e574ae3cc91efd93860ad9e3d99378 Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Wed, 24 Jan 2024 20:51:29 +0530 Subject: [PATCH 05/21] Renamed parameters in Example DAGs, unit and System tests of teradata_to_teradata operator --- ...e_teradata_to_teradata_transfer_operator.py | 8 ++++---- .../teradata/transfers/teradata_to_teradata.py | 1 - .../transfers/test_teradata_to_teradata.py | 18 +++++++++--------- .../example_teradata_to_teradata_transfer.py | 8 ++++---- 4 files changed, 17 insertions(+), 18 deletions(-) diff --git a/airflow/providers/teradata/example_dags/example_teradata_to_teradata_transfer_operator.py b/airflow/providers/teradata/example_dags/example_teradata_to_teradata_transfer_operator.py index 49c97485dcf715..df4e6521d6064a 100644 --- a/airflow/providers/teradata/example_dags/example_teradata_to_teradata_transfer_operator.py +++ b/airflow/providers/teradata/example_dags/example_teradata_to_teradata_transfer_operator.py @@ -109,11 +109,11 @@ transfer_data = TeradataToTeradataOperator( task_id="transfer_data", - teradata_destination_conn_id="teradata_default", + dest_teradata_conn_id="teradata_default", destination_table="my_users_dest", - teradata_source_conn_id="teradata_default", - source_sql="select * from my_users_src", - source_sql_params={}, + source_teradata_conn_id="teradata_default", + sql="select * from my_users_src", + sql_params={}, rows_chunk=2, ) diff --git a/airflow/providers/teradata/transfers/teradata_to_teradata.py b/airflow/providers/teradata/transfers/teradata_to_teradata.py index 9889bee7e1b77e..f1030ccce8915e 100644 --- a/airflow/providers/teradata/transfers/teradata_to_teradata.py +++ b/airflow/providers/teradata/transfers/teradata_to_teradata.py @@ -85,7 +85,6 @@ def _execute(self, src_hook, dest_hook, context) -> None: self.log.info("Finished data transfer.") cursor.close() - def execute(self, context: Context) -> None: src_hook = TeradataHook(teradata_conn_id=self.source_teradata_conn_id) dest_hook = TeradataHook(teradata_conn_id=self.dest_teradata_conn_id) diff --git a/tests/providers/teradata/transfers/test_teradata_to_teradata.py b/tests/providers/teradata/transfers/test_teradata_to_teradata.py index fc2b45561fac1d..496ca1dad35b5c 100644 --- a/tests/providers/teradata/transfers/test_teradata_to_teradata.py +++ b/tests/providers/teradata/transfers/test_teradata_to_teradata.py @@ -26,11 +26,11 @@ class TestTeradataToTeradataTransfer: def test_execute(self): - teradata_destination_conn_id = "teradata_destination_conn_id" + dest_teradata_conn_id = "dest_teradata_conn_id" destination_table = "destination_table" - teradata_source_conn_id = "teradata_source_conn_id" - source_sql = (r"""select DATE where DATE > {{ source_sql_params.ref_date }};""",) - source_sql_params = {"ref_date": "2018-01-01"} + source_teradata_conn_id = "source_teradata_conn_id" + sql = (r"""select DATE where DATE > {{ source_sql_params.ref_date }};""",) + sql_params = {"ref_date": "2018-01-01"} rows_chunk = 5000 cursor_description = [ ["user_id", Decimal, None, 8, 10, 0, False], @@ -48,11 +48,11 @@ def test_execute(self): td_transfer_op = TeradataToTeradataOperator( task_id="transfer_data", - teradata_destination_conn_id=teradata_destination_conn_id, + dest_teradata_conn_id=dest_teradata_conn_id, destination_table=destination_table, - teradata_source_conn_id=teradata_source_conn_id, - source_sql=source_sql, - source_sql_params=source_sql_params, + source_teradata_conn_id=source_teradata_conn_id, + sql=sql, + sql_params=sql_params, rows_chunk=rows_chunk, ) @@ -60,7 +60,7 @@ def test_execute(self): assert mock_src_hook.get_conn.called assert mock_src_conn.cursor.called - mock_cursor.execute.assert_called_once_with(source_sql, source_sql_params) + mock_cursor.execute.assert_called_once_with(sql, sql_params) calls = [ mock.call(rows_chunk), diff --git a/tests/system/providers/teradata/example_teradata_to_teradata_transfer.py b/tests/system/providers/teradata/example_teradata_to_teradata_transfer.py index aac9e151a9c15b..8727715f8a6495 100644 --- a/tests/system/providers/teradata/example_teradata_to_teradata_transfer.py +++ b/tests/system/providers/teradata/example_teradata_to_teradata_transfer.py @@ -120,11 +120,11 @@ # [START teradata_to_teradata_transfer_operator_howto_guide_transfer_data] transfer_data = TeradataToTeradataOperator( task_id="transfer_data", - teradata_destination_conn_id="teradata_default", + dest_teradata_conn_id="teradata_default", destination_table="my_users_dest", - teradata_source_conn_id="teradata_default", - source_sql="select * from my_users_src", - source_sql_params={}, + source_teradata_conn_id="teradata_default", + sql="select * from my_users_src", + sql_params={}, rows_chunk=2, ) # [END teradata_to_teradata_transfer_operator_howto_guide_transfer_data] From d02d16fddda830f0b528e89fad2096274e4f7d83 Mon Sep 17 00:00:00 2001 From: SatishChGit Date: Fri, 2 Feb 2024 16:56:24 +0530 Subject: [PATCH 06/21] Update airflow/providers/teradata/operators/teradata.py Co-authored-by: Josh Fell <48934154+josh-fell@users.noreply.github.com> --- airflow/providers/teradata/operators/teradata.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/airflow/providers/teradata/operators/teradata.py b/airflow/providers/teradata/operators/teradata.py index 8a496768fc7a4b..8cb1d6769d42d0 100644 --- a/airflow/providers/teradata/operators/teradata.py +++ b/airflow/providers/teradata/operators/teradata.py @@ -29,8 +29,9 @@ class TeradataOperator(SQLExecuteQueryOperator): Executes sql statements in the Teradata SQL Database using teradatasql jdbc driver .. seealso:: - For more information on how to use this operator, take a look at the guide: - :ref:`howto/operator:TeradataOperator` + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:TeradataOperator` + :param sql: the SQL query to be executed as a single string, or a list of str (sql statements) :param conn_id: reference to a predefined database :param autocommit: if True, each command is automatically committed.(default value: False) From 6a8960c7c91b5d658e7acd46ad71ba5958443cec Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Sat, 3 Feb 2024 12:05:30 +0530 Subject: [PATCH 07/21] Addressed review comments --- contributing-docs/12_airflow_dependencies_and_extras.rst | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/contributing-docs/12_airflow_dependencies_and_extras.rst b/contributing-docs/12_airflow_dependencies_and_extras.rst index c9124d1ccbd6f1..72de42648cc21d 100644 --- a/contributing-docs/12_airflow_dependencies_and_extras.rst +++ b/contributing-docs/12_airflow_dependencies_and_extras.rst @@ -211,7 +211,8 @@ microsoft-mssql, microsoft-psrp, microsoft-winrm, mongo, mssql, mysql, neo4j, od openfaas, openlineage, opensearch, opsgenie, oracle, otel, pagerduty, pandas, papermill, password, pgvector, pinecone, pinot, postgres, presto, rabbitmq, redis, s3, s3fs, salesforce, samba, saml, segment, sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd, -tableau, tabular, telegram, trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex, zendesk +tableau, tabular, telegram, teradata, trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex, +zendesk .. END REGULAR EXTRAS HERE From 771d7532c4b197009941c5dd021d08149c91fe7d Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Sat, 3 Feb 2024 13:43:12 +0530 Subject: [PATCH 08/21] Squashed commit of the following: commit 4161354489fa9443c940310b8e5590aa876d6e19 Author: Satish Ch Date: Sat Feb 3 00:10:56 2024 -0800 Removed not relavent changes commit bf2e8bee95dd50dd3a1ad2a15e14e802a49e282a Author: Satish Ch Date: Fri Feb 2 23:41:01 2024 -0800 static check applied commit 14b61f8d23d658d8ff5bec07b97bdcde24745cb9 Author: Satish Ch Date: Sat Feb 3 12:33:53 2024 +0530 Review comments addressed --- .../teradata/example_dags/__init__.py | 17 -- .../example_dags/example_teradata_hook.py | 126 -------------- .../example_dags/example_teradata_operator.py | 132 --------------- ..._teradata_to_teradata_transfer_operator.py | 158 ------------------ airflow/providers/teradata/hooks/teradata.py | 5 +- .../index.rst | 3 +- .../operators/teradata.rst | 6 +- .../operators/teradata_to_teradata.rst | 6 +- .../providers/teradata/hooks/test_teradata.py | 10 +- .../teradata/operators/test_teradata.py | 16 +- 10 files changed, 18 insertions(+), 461 deletions(-) delete mode 100644 airflow/providers/teradata/example_dags/__init__.py delete mode 100644 airflow/providers/teradata/example_dags/example_teradata_hook.py delete mode 100644 airflow/providers/teradata/example_dags/example_teradata_operator.py delete mode 100644 airflow/providers/teradata/example_dags/example_teradata_to_teradata_transfer_operator.py diff --git a/airflow/providers/teradata/example_dags/__init__.py b/airflow/providers/teradata/example_dags/__init__.py deleted file mode 100644 index 217e5db9607827..00000000000000 --- a/airflow/providers/teradata/example_dags/__init__.py +++ /dev/null @@ -1,17 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. diff --git a/airflow/providers/teradata/example_dags/example_teradata_hook.py b/airflow/providers/teradata/example_dags/example_teradata_hook.py deleted file mode 100644 index f071076b8c5faa..00000000000000 --- a/airflow/providers/teradata/example_dags/example_teradata_hook.py +++ /dev/null @@ -1,126 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -""" -Example Airflow DAG showing usage of Teradata Airflow Provider hook interacting with Teradata SQL DB. - -This DAG assumes Airflow Connection with connection id `teradata_default` already exists in locally. -It shows how to establish connection with Teradata SQL Database server and how to run queries on it. -""" -from __future__ import annotations - -from contextlib import closing -from datetime import datetime, timedelta - -import teradatasql - -from airflow import DAG -from airflow.operators.python import PythonOperator -from airflow.providers.teradata.hooks.teradata import TeradataHook - -# Constants used for below example DAG -TEST_SQL = "SELECT DATE;" -CONN_ID = "teradata_default" - -# [START howto_hook_teradata] - - -def teradata_hook_usage(): - # This creates connection with Teradata database using default connection id 'teradata_default' - tdh = TeradataHook() - - # This creates connection with Teradata database overriding database name from what is specified - # in teradata_default connection id - tdh = TeradataHook(teradata_conn_id=CONN_ID, database="database_name") - - # Verify connection to database server works - tdh.test_connection() - - # Check connection config details - tdh.get_uri() - # print(tdh.get_uri()) - - # This method gets back a TeradataConnection object which is created using teradatasql client internally - conn = tdh.get_conn() - - # This method returns sqlalchemy engine connection object that points to Teradata database - tdh.get_sqlalchemy_engine() - - # Execute select queries directly using TeradataHook - tdh.get_records(sql=TEST_SQL) - tdh.get_first(sql=TEST_SQL) - - # Execute any other modification queries using TeradataHook - tdh.run(sql=TEST_SQL) - tdh.run(sql=["SELECT 1;", "SELECT 2;"]) - - # Execute insert queries - # rows = [('User5',), ('User6',)] - # target_fields = ["user_name"] - # res = tdh.insert_rows(table="my_users", rows=rows, target_fields=target_fields) - - # Get cursor object from connection to manually run queries and get results - # Read more about using teradatasql connection here: https://pypi.org/project/teradatasql/ - cursor = tdh.get_cursor() - cursor.execute(TEST_SQL).fetchall() - [d[0] for d in cursor.description] - try: - # print("Test executing queries in Teradata database...") - with closing(tdh.get_conn()) as conn, closing(conn.cursor()) as cursor: - cursor.execute("SELECT DATE") - conn.commit() - print("Ran the query on Teradata database") - except (teradatasql.OperationalError, teradatasql.Warning): - # print("Error running query on Teradata database") - raise - - # Get query result as pandas dataframe - tdh.get_pandas_df(sql=TEST_SQL) - - # Get query results as chunks of rows as pandas generator - gen = tdh.get_pandas_df_by_chunks(sql=TEST_SQL, chunksize=2) - while True: - try: - next(gen) - except StopIteration: - break - # print(rows) - - # Saving data to a staging table using pandas to_sql - # conn = tdh.get_sqlalchemy_engine() - # df.to_sql("temp_my_users", con=conn, if_exists="replace") - - -# [END howto_hook_teradata] - - -with DAG( - dag_id="example_teradata_hook", - description="""Sample usage of the TeradataHook airflow provider module""", - start_date=datetime(2023, 1, 1), - max_active_runs=1, - max_active_tasks=3, - catchup=False, - tags=["example"], - default_args={ - "retries": 2, - "retry_delay": timedelta(minutes=6), - }, -) as dag: - show_teradata_hook_usage = PythonOperator( - task_id="show_teradata_hook_usage", python_callable=teradata_hook_usage - ) diff --git a/airflow/providers/teradata/example_dags/example_teradata_operator.py b/airflow/providers/teradata/example_dags/example_teradata_operator.py deleted file mode 100644 index 6b868c1bda06e3..00000000000000 --- a/airflow/providers/teradata/example_dags/example_teradata_operator.py +++ /dev/null @@ -1,132 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -""" -Example Airflow DAG to show basic CRUD operation on teradata database using TeradataOperator - -This DAG assumes Airflow Connection with connection id `teradata_default` already exists in locally. -It shows how to run queries as tasks in airflow dags using TeradataOperator.. -""" -from __future__ import annotations - -from datetime import datetime - -import pytest - -from airflow import DAG -from airflow.models.baseoperator import chain - -try: - from airflow.providers.teradata.operators.teradata import TeradataOperator -except ImportError: - pytest.skip("Teradata provider apache-airflow-provider-teradata not available", allow_module_level=True) - - -CONN_ID = "teradata_default" - - -with DAG( - dag_id="example_teradata_operator", - max_active_runs=1, - max_active_tasks=3, - catchup=False, - start_date=datetime(2023, 1, 1), -) as dag: - # [START howto_teradata_operator] - - create = TeradataOperator( - task_id="table_create", - conn_id=CONN_ID, - sql=""" - CREATE TABLE my_users, - FALLBACK ( - user_id decimal(10,0) NOT NULL GENERATED ALWAYS AS IDENTITY ( - START WITH 1 - INCREMENT BY 1 - MINVALUE 1 - MAXVALUE 2147483647 - NO CYCLE), - user_name VARCHAR(30) - ) PRIMARY INDEX (user_id); - """, - ) - - insert = TeradataOperator( - task_id="insert_rows", - conn_id=CONN_ID, - sql=""" - INSERT INTO my_users(user_name) VALUES ('User1'); - INSERT INTO my_users(user_name) VALUES ('User2'); - INSERT INTO my_users(user_name) VALUES ('User3'); - INSERT INTO my_users(user_name) VALUES ('User4'); - INSERT INTO my_users(user_name) VALUES ('User5'); - INSERT INTO my_users(user_name) VALUES ('User6'); - INSERT INTO my_users(user_name) VALUES ('User7'); - INSERT INTO my_users(user_name) VALUES ('User8'); - INSERT INTO my_users(user_name) VALUES ('User9'); - INSERT INTO my_users(user_name) VALUES ('User10'); - """, - ) - - update = TeradataOperator( - task_id="update_data", - conn_id=CONN_ID, - sql=""" - UPDATE my_users SET user_name = 'User123' WHERE user_name = 'User1'; - """, - ) - - read_data = TeradataOperator( - task_id="read_data", - conn_id=CONN_ID, - sql=""" - SELECT TOP 10 * from my_users order by user_id desc; - """, - ) - - delete = TeradataOperator( - task_id="delete_rows", - conn_id=CONN_ID, - sql=""" - DELETE FROM my_users ALL; - """, - ) - - alter_table = TeradataOperator( - task_id="alter_table", - conn_id=CONN_ID, - sql=""" - ALTER TABLE my_users - ADD gender CHAR(1) DEFAULT 'M', - ADD birth_date DATE FORMAT 'YYYY-MM-DD' NOT NULL DEFAULT DATE '2023-01-01'; - """, - ) - - drop = TeradataOperator( - task_id="drop_table", - conn_id=CONN_ID, - sql=""" - DROP TABLE my_users; - """, - ) - - chain(create, insert, update, read_data, delete, alter_table, drop) - - # Make sure create was done before deleting table - create >> drop - - # [END howto_teradata_operator] diff --git a/airflow/providers/teradata/example_dags/example_teradata_to_teradata_transfer_operator.py b/airflow/providers/teradata/example_dags/example_teradata_to_teradata_transfer_operator.py deleted file mode 100644 index df4e6521d6064a..00000000000000 --- a/airflow/providers/teradata/example_dags/example_teradata_to_teradata_transfer_operator.py +++ /dev/null @@ -1,158 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -""" -Example Airflow DAG to show usage of teradata to teradata transfer operator - -The transfer operator connects to source teradata server, runs query to fetch data from source -and inserts that data into destination teradata database server. It assumes tables already exists. -The example DAG below assumes Airflow Connection with connection id `teradata_default` already exists. -It creates sample my_users table at source and destination, sets up sample data at source and then -runs transfer operator to copy data to corresponding table on destination server. -""" -from __future__ import annotations - -from datetime import datetime - -import pytest - -from airflow import DAG -from airflow.models.baseoperator import chain - -try: - from airflow.providers.teradata.operators.teradata import TeradataOperator - from airflow.providers.teradata.transfers.teradata_to_teradata import TeradataToTeradataOperator -except ImportError: - pytest.skip("Teradata provider apache-airflow-provider-teradata not available", allow_module_level=True) - - -CONN_ID = "teradata_default" - - -with DAG( - dag_id="example_teradata_to_teradata_transfer_operator", - max_active_runs=1, - max_active_tasks=3, - catchup=False, - start_date=datetime(2023, 1, 1), -) as dag: - # [START howto_transfer_operator_teradata_to_teradata] - - create_src_table = TeradataOperator( - task_id="create_src_table", - conn_id=CONN_ID, - sql=""" - CREATE TABLE my_users_src, - FALLBACK ( - user_id decimal(10,0) NOT NULL GENERATED ALWAYS AS IDENTITY ( - START WITH 1 - INCREMENT BY 1 - MINVALUE 1 - MAXVALUE 2147483647 - NO CYCLE), - user_name VARCHAR(30), - gender CHAR(1) DEFAULT 'M', - birth_date DATE FORMAT 'YYYY-MM-DD' NOT NULL DEFAULT DATE '2023-01-01' - ) PRIMARY INDEX (user_id); - """, - ) - - create_dest_table = TeradataOperator( - task_id="create_dest_table", - conn_id=CONN_ID, - sql=""" - CREATE TABLE my_users_dest, - FALLBACK ( - user_id decimal(10,0) NOT NULL GENERATED ALWAYS AS IDENTITY ( - START WITH 1 - INCREMENT BY 1 - MINVALUE 1 - MAXVALUE 2147483647 - NO CYCLE), - user_name VARCHAR(30), - gender CHAR(1) DEFAULT 'M', - birth_date DATE FORMAT 'YYYY-MM-DD' NOT NULL DEFAULT DATE '2023-01-01' - ) PRIMARY INDEX (user_id); - """, - ) - insert_data_src = TeradataOperator( - task_id="insert_data_src", - conn_id=CONN_ID, - sql=""" - INSERT INTO my_users_src(user_name) VALUES ('User1'); - INSERT INTO my_users_src(user_name) VALUES ('User2'); - INSERT INTO my_users_src(user_name) VALUES ('User3'); - """, - ) - - read_data_src = TeradataOperator( - task_id="read_data_src", - conn_id=CONN_ID, - sql=""" - SELECT TOP 10 * from my_users_src order by user_id desc; - """, - ) - - transfer_data = TeradataToTeradataOperator( - task_id="transfer_data", - dest_teradata_conn_id="teradata_default", - destination_table="my_users_dest", - source_teradata_conn_id="teradata_default", - sql="select * from my_users_src", - sql_params={}, - rows_chunk=2, - ) - - read_data_dest = TeradataOperator( - task_id="read_data_dest", - conn_id=CONN_ID, - sql=""" - SELECT TOP 10 * from my_users_dest order by user_id desc; - """, - ) - - drop_src_table = TeradataOperator( - task_id="drop_src_table", - conn_id=CONN_ID, - sql=""" - DROP TABLE my_users_src; - """, - ) - - drop_dest_table = TeradataOperator( - task_id="drop_dest_table", - conn_id=CONN_ID, - sql=""" - DROP TABLE my_users_dest; - """, - ) - - chain( - create_src_table, - create_dest_table, - insert_data_src, - read_data_src, - transfer_data, - read_data_dest, - drop_src_table, - drop_dest_table, - ) - - # Make sure create was done before deleting table - create_src_table >> drop_src_table - create_dest_table >> drop_dest_table - # [END howto_transfer_operator_teradata_to_teradata] diff --git a/airflow/providers/teradata/hooks/teradata.py b/airflow/providers/teradata/hooks/teradata.py index a53f1e4b71f2eb..f295168fb8be60 100644 --- a/airflow/providers/teradata/hooks/teradata.py +++ b/airflow/providers/teradata/hooks/teradata.py @@ -200,10 +200,11 @@ def get_ui_field_behaviour() -> dict: "host": "Database Server URL", "schema": "Database Name", "login": "Username", - "password": "Password", }, "placeholders": { - "extra": json.dumps({"example_parameter": "parameter"}, indent=4), + "extra": json.dumps( + {"tmode": "TERA", "sslmode": "verify-ca", "sslca": "/tmp/server-ca.pem"}, indent=4 + ), "login": "dbc", "password": "dbc", }, diff --git a/docs/apache-airflow-providers-teradata/index.rst b/docs/apache-airflow-providers-teradata/index.rst index 13dd7a4e9b79e2..55dd27f08a52c2 100644 --- a/docs/apache-airflow-providers-teradata/index.rst +++ b/docs/apache-airflow-providers-teradata/index.rst @@ -57,7 +57,6 @@ :maxdepth: 1 :caption: Resources - Example DAGs PyPI Repository Installing from sources @@ -103,6 +102,8 @@ PIP package Version required ======================================= ================== ``apache-airflow`` ``>=2.6.0`` ``apache-airflow-providers-common-sql`` ``>=1.3.1`` +``teradatasqlalchemy`` ``>=17.20.0.0`` +``teradatasql`` ``>=17.20.0.28`` ======================================= ================== Cross provider package dependencies diff --git a/docs/apache-airflow-providers-teradata/operators/teradata.rst b/docs/apache-airflow-providers-teradata/operators/teradata.rst index 8b902ad9afcc0c..4d3967e017bd7d 100644 --- a/docs/apache-airflow-providers-teradata/operators/teradata.rst +++ b/docs/apache-airflow-providers-teradata/operators/teradata.rst @@ -30,7 +30,7 @@ To execute arbitrary SQL in an Teradata, use the An example of executing a simple query is as follows: -.. exampleinclude:: /../../airflow/providers/teradata/example_dags/example_teradata_operator.py +.. exampleinclude:: /../../tests/system/providers/teradata/example_teradata.py :language: python - :start-after: [START howto_teradata_operator] - :end-before: [END howto_teradata_operator] + :start-after: [START teradata_operator_howto_guide] + :end-before: [END teradata_operator_howto_guide] diff --git a/docs/apache-airflow-providers-teradata/operators/teradata_to_teradata.rst b/docs/apache-airflow-providers-teradata/operators/teradata_to_teradata.rst index 9c010be5a61a1e..20c76c2d9bc825 100644 --- a/docs/apache-airflow-providers-teradata/operators/teradata_to_teradata.rst +++ b/docs/apache-airflow-providers-teradata/operators/teradata_to_teradata.rst @@ -32,7 +32,7 @@ To transfer data between two Teradata instances, use the An example usage of the TeradataToTeradataOperator is as follows: -.. exampleinclude:: /../../airflow/providers/teradata/example_dags/example_teradata_to_teradata_transfer_operator.py +.. exampleinclude:: /../../tests/system/providers/teradata/example_teradata_to_teradata_transfer.py :language: python - :start-after: [START howto_transfer_operator_teradata_to_teradata] - :end-before: [END howto_transfer_operator_teradata_to_teradata] + :start-after: [START teradata_to_teradata_transfer_operator_howto_guide] + :end-before: [END teradata_to_teradata_transfer_operator_howto_guide] diff --git a/tests/providers/teradata/hooks/test_teradata.py b/tests/providers/teradata/hooks/test_teradata.py index 7d0576f0698d5c..8f7725cbdaa8c2 100644 --- a/tests/providers/teradata/hooks/test_teradata.py +++ b/tests/providers/teradata/hooks/test_teradata.py @@ -25,14 +25,8 @@ from airflow.models import Connection -try: - from airflow.providers.teradata.hooks.teradata import TeradataHook -except ImportError: - pytest.skip( - "Airflow Provider for Teradata not available, unable to import dependency " - "airflow.providers.teradata.hooks.teradata.TeradataHook", - allow_module_level=True, - ) +from airflow.providers.teradata.hooks.teradata import TeradataHook + class TestTeradataHook: diff --git a/tests/providers/teradata/operators/test_teradata.py b/tests/providers/teradata/operators/test_teradata.py index 4dcaf7a924076f..ca9e9fb44cf866 100644 --- a/tests/providers/teradata/operators/test_teradata.py +++ b/tests/providers/teradata/operators/test_teradata.py @@ -20,17 +20,13 @@ from unittest import mock from unittest.mock import MagicMock, Mock -import pytest - from airflow.models.dag import DAG from airflow.providers.common.sql.hooks.sql import fetch_all_handler from airflow.utils import timezone -try: - from airflow.providers.teradata.hooks.teradata import TeradataHook - from airflow.providers.teradata.operators.teradata import TeradataOperator -except ImportError: - pytest.skip("Teradata not available", allow_module_level=True) +from airflow.providers.teradata.hooks.teradata import TeradataHook +from airflow.providers.teradata.operators.teradata import TeradataOperator + from airflow.exceptions import AirflowException @@ -47,12 +43,10 @@ def setup_method(self): @mock.patch("airflow.providers.common.sql.operators.sql.SQLExecuteQueryOperator.get_db_hook") def test_get_hook_from_conn(self, mock_get_db_hook): """ - :class:`~.MsSqlOperator` should use the hook returned by :meth:`airflow.models.Connection.get_hook` + :class:`~.TeradataOperator` should use the hook returned by :meth:`airflow.models.Connection.get_hook` if one is returned. - This behavior is necessary in order to support usage of :class:`~.OdbcHook` with this operator. - - Specifically we verify here that :meth:`~.MsSqlOperator.get_hook` returns the hook returned from a + Specifically we verify here that :meth:`~.TeradataOperator.get_hook` returns the hook returned from a call of ``get_hook`` on the object returned from :meth:`~.BaseHook.get_connection`. """ mock_hook = MagicMock() From ef638436799c741259786532f78988673e808b54 Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Sat, 3 Feb 2024 13:50:51 +0530 Subject: [PATCH 09/21] Added database param to documentation --- airflow/providers/teradata/hooks/teradata.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/providers/teradata/hooks/teradata.py b/airflow/providers/teradata/hooks/teradata.py index f295168fb8be60..077009afdc2c73 100644 --- a/airflow/providers/teradata/hooks/teradata.py +++ b/airflow/providers/teradata/hooks/teradata.py @@ -42,6 +42,7 @@ class TeradataHook(DbApiHook): See :doc:` docs/apache-airflow-providers-teradata/connections/teradata.rst` for full documentation. :param args: passed to DbApiHook + :param database: The Teradata database to connect to. :param kwargs: passed to DbApiHook From 5e36515cc954a44756f2b3de83d9b6580a1a1b2f Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Sat, 3 Feb 2024 14:11:49 +0530 Subject: [PATCH 10/21] Addressed review comments --- airflow/providers/teradata/operators/teradata.py | 3 +++ airflow/providers/teradata/transfers/teradata_to_teradata.py | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/airflow/providers/teradata/operators/teradata.py b/airflow/providers/teradata/operators/teradata.py index 8cb1d6769d42d0..3e21189e0f83f7 100644 --- a/airflow/providers/teradata/operators/teradata.py +++ b/airflow/providers/teradata/operators/teradata.py @@ -36,6 +36,9 @@ class TeradataOperator(SQLExecuteQueryOperator): :param conn_id: reference to a predefined database :param autocommit: if True, each command is automatically committed.(default value: False) :param parameters: (optional) the parameters to render the SQL query with. + :param host: The Teradata instance hostname to connect to. + :param schema: The Teradata database to connect to. + :param login: Username to connect """ template_fields: Sequence[str] = ( diff --git a/airflow/providers/teradata/transfers/teradata_to_teradata.py b/airflow/providers/teradata/transfers/teradata_to_teradata.py index f1030ccce8915e..dad8913ee3b774 100644 --- a/airflow/providers/teradata/transfers/teradata_to_teradata.py +++ b/airflow/providers/teradata/transfers/teradata_to_teradata.py @@ -82,7 +82,7 @@ def _execute(self, src_hook, dest_hook, context) -> None: self.destination_table, rows, target_fields=target_fields, commit_every=self.rows_chunk ) rows_total += len(rows) - self.log.info("Finished data transfer.") + self.log.info("Finished data transfer. Total number of rows transferred - %s", rows_total) cursor.close() def execute(self, context: Context) -> None: From 2c3a3c867be6bcbd892790a9aa7f461a8b113d00 Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Mon, 5 Feb 2024 22:12:41 +0530 Subject: [PATCH 11/21] Squashed commit of the following: commit 96df1feaaf372adb492369f703c7e74f2e46f814 Author: Satish Ch Date: Mon Feb 5 21:46:15 2024 +0530 documentation updated commit d7dcbf7972ce37ad6f236c61398dc4566169d1aa Author: Satish Ch Date: Mon Feb 5 21:37:24 2024 +0530 Update example_teradata.py commit 2644849d7060d5878a758cc743a1a63cbeadbd1e Merge: 8da3658b00 c19a9def71 Author: Satish Ch Date: Mon Feb 5 21:31:56 2024 +0530 Merge branch 'pr_teradata_release_1.0.0' of https://github.com/Teradata/airflow into pr_teradata_release_1.0.0 commit 8da3658b00c8602c071044505eb58f128c607cd1 Author: Satish Ch Date: Mon Feb 5 21:31:51 2024 +0530 format applied on system example commit c19a9def7108ba51895010d3dfe2d0564e7e1cbc Author: Satish Ch Date: Mon Feb 5 07:54:26 2024 -0800 format appied commit 633041d24760b15f1f34d64ad3358a1f420386ed Author: Satish Ch Date: Mon Feb 5 21:20:19 2024 +0530 Update operators.rst commit 004c50135c5a580b01e567b6950257a0f181709e Author: Satish Ch Date: Mon Feb 5 21:06:55 2024 +0530 task order modified commit 654ca06c9cf9e39cbfaee4300555327a5a02e639 Merge: 19240da780 30e4b28f42 Author: Satish Ch Date: Mon Feb 5 20:52:27 2024 +0530 Merge branch 'pr_teradata_release_1.0.0' of https://github.com/Teradata/airflow into pr_teradata_release_1.0.0 commit 19240da780152a5917e452f2fd5d2d27d58a4d17 Author: Satish Ch Date: Mon Feb 5 20:52:21 2024 +0530 Documentation improved for Teradata Operator commit 30e4b28f4229c3c7b56ea14bf2ef664e9f882450 Author: Satish Ch Date: Mon Feb 5 06:38:52 2024 -0800 static check change commit ef730c7bb6c9c9ed04199368e248ba233888ee9b Author: Satish Ch Date: Mon Feb 5 19:48:11 2024 +0530 Update teradata.py commit d76a93a7f6c88ef0dcbc250f1186522a033d8bd6 Author: Satish Ch Date: Mon Feb 5 19:34:52 2024 +0530 new tasks added for new schema commit 5800736de8b9cc5bba496ed929af70dbf9c51236 Author: Satish Ch Date: Mon Feb 5 19:12:36 2024 +0530 if schema defined overriding this with explict value provided commit 932b8f16a90335acbc4387ac06a7ad993dd5126f Author: Satish Ch Date: Mon Feb 5 19:07:04 2024 +0530 Removed login and password as it is antipattern as per PR review comments commit 77352d8a592851f1c482c16f2f74adb98687f4b5 Author: Satish Ch Date: Mon Feb 5 05:18:47 2024 -0800 format applied commit ab1f8b297ed815a2168aaa80bbc322aede82deea Author: Satish Ch Date: Mon Feb 5 17:08:49 2024 +0530 Review comments addressed commit c1b9cc9c527c7127c1731198edabfb0d9be078ae Merge: df42832fd4 5e36515cc9 Author: Satish Ch Date: Mon Feb 5 16:50:25 2024 +0530 Merge branch 'teradata_release_1.0.0' into pr_teradata_release_1.0.0 commit df42832fd493b91f25f4800dcbe68a4e4af3158e Author: Satish Ch Date: Sat Feb 3 01:12:47 2024 -0800 static format applied commit 4161354489fa9443c940310b8e5590aa876d6e19 Author: Satish Ch Date: Sat Feb 3 00:10:56 2024 -0800 Removed not relavent changes commit bf2e8bee95dd50dd3a1ad2a15e14e802a49e282a Author: Satish Ch Date: Fri Feb 2 23:41:01 2024 -0800 static check applied commit 14b61f8d23d658d8ff5bec07b97bdcde24745cb9 Author: Satish Ch Date: Sat Feb 3 12:33:53 2024 +0530 Review comments addressed --- airflow/providers/teradata/hooks/teradata.py | 6 +- .../providers/teradata/operators/teradata.py | 13 +- .../operators.rst | 4 +- .../operators/teradata.rst | 86 ++++++++++++- .../operators/teradata_to_teradata.rst | 14 ++- .../redirects.txt | 1 - pyproject.toml | 2 +- .../providers/teradata/hooks/test_teradata.py | 2 - .../teradata/operators/test_teradata.py | 17 +-- .../providers/teradata/example_teradata.py | 117 +++++++++++++----- .../example_teradata_to_teradata_transfer.py | 5 - 11 files changed, 194 insertions(+), 73 deletions(-) delete mode 100644 docs/apache-airflow-providers-teradata/redirects.txt diff --git a/airflow/providers/teradata/hooks/teradata.py b/airflow/providers/teradata/hooks/teradata.py index 077009afdc2c73..fc3f3d521a03eb 100644 --- a/airflow/providers/teradata/hooks/teradata.py +++ b/airflow/providers/teradata/hooks/teradata.py @@ -18,7 +18,7 @@ """A Airflow Hook for interacting with Teradata SQL Server.""" from __future__ import annotations -from typing import TYPE_CHECKING, Any, TypeVar +from typing import TYPE_CHECKING, Any import sqlalchemy import teradatasql @@ -26,7 +26,6 @@ from airflow.providers.common.sql.hooks.sql import DbApiHook -T = TypeVar("T") if TYPE_CHECKING: from airflow.models.connection import Connection @@ -39,7 +38,8 @@ class TeradataHook(DbApiHook): Teradata DB Server URL, username, password and database name are fetched from the predefined connection config connection_id. It raises an airflow error if the given connection id doesn't exist. - See :doc:` docs/apache-airflow-providers-teradata/connections/teradata.rst` for full documentation. + .. seealso:: + - :ref:`Teradata API connection ` :param args: passed to DbApiHook :param database: The Teradata database to connect to. diff --git a/airflow/providers/teradata/operators/teradata.py b/airflow/providers/teradata/operators/teradata.py index 3e21189e0f83f7..ae87fd785487f4 100644 --- a/airflow/providers/teradata/operators/teradata.py +++ b/airflow/providers/teradata/operators/teradata.py @@ -28,6 +28,7 @@ class TeradataOperator(SQLExecuteQueryOperator): General Teradata Operator to execute queries on Teradata Database. Executes sql statements in the Teradata SQL Database using teradatasql jdbc driver + .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:TeradataOperator` @@ -36,9 +37,7 @@ class TeradataOperator(SQLExecuteQueryOperator): :param conn_id: reference to a predefined database :param autocommit: if True, each command is automatically committed.(default value: False) :param parameters: (optional) the parameters to render the SQL query with. - :param host: The Teradata instance hostname to connect to. :param schema: The Teradata database to connect to. - :param login: Username to connect """ template_fields: Sequence[str] = ( @@ -52,19 +51,13 @@ class TeradataOperator(SQLExecuteQueryOperator): def __init__( self, conn_id: str = TeradataHook.default_conn_name, - host: str | None = None, schema: str | None = None, - login: str | None = None, - password: str | None = None, **kwargs, ) -> None: - if any([host, schema, login, password]): + if schema: hook_params = kwargs.pop("hook_params", {}) kwargs["hook_params"] = { - "host": host, - "schema": schema, - "login": login, - "password": password, + "database": schema, **hook_params, } super().__init__(**kwargs) diff --git a/docs/apache-airflow-providers-microsoft-mssql/operators.rst b/docs/apache-airflow-providers-microsoft-mssql/operators.rst index 0dd55172150c20..e2491c2d117f90 100644 --- a/docs/apache-airflow-providers-microsoft-mssql/operators.rst +++ b/docs/apache-airflow-providers-microsoft-mssql/operators.rst @@ -43,7 +43,7 @@ An example usage of the MsSqlOperator is as follows: :start-after: [START howto_operator_mssql] :end-before: [END howto_operator_mssql] -You can also use an external file to execute the SQL commands. Script folder must be at the same level as DAG.py file. +You can also use an external file to execute the SQL commands. External sql file must be at the same level as DAG.py file. This way you can easily maintain the SQL queries separated from the code. .. exampleinclude:: /../../tests/system/providers/microsoft/mssql/example_mssql.py @@ -54,7 +54,7 @@ This way you can easily maintain the SQL queries separated from the code. Your ``dags/create_table.sql`` should look like this: -.. code-block::sql +.. code-block:: sql -- create Users table CREATE TABLE Users ( diff --git a/docs/apache-airflow-providers-teradata/operators/teradata.rst b/docs/apache-airflow-providers-teradata/operators/teradata.rst index 4d3967e017bd7d..f0e0a299b120d7 100644 --- a/docs/apache-airflow-providers-teradata/operators/teradata.rst +++ b/docs/apache-airflow-providers-teradata/operators/teradata.rst @@ -22,13 +22,91 @@ TeradataOperator The purpose of TeradataOperator is to define tasks involving interactions with the Teradata. -Execute SQL in an Teradata ---------------------------------- - To execute arbitrary SQL in an Teradata, use the :class:`~airflow.providers.teradata.operators.teradata.TeradataOperator`. -An example of executing a simple query is as follows: +Common Database Operations with TeradataOperator +------------------------------------------------ + +Creating a Teradata database table +---------------------------------- + +An example usage of the TeradataOperator is as follows: + +.. exampleinclude:: /../../tests/system/providers/teradata/example_teradata.py + :language: python + :start-after: [START teradata_operator_howto_guide_create_table] + :end-before: [END teradata_operator_howto_guide_create_table] + +You can also use an external file to execute the SQL commands. Script folder must be at the same level as DAG.py file. +This way you can easily maintain the SQL queries separated from the code. + +.. exampleinclude:: /../../tests/system/providers/teradata/example_teradata.py + :language: python + :start-after: [START teradata_operator_howto_guide_create_table_from_external_file] + :end-before: [END teradata_operator_howto_guide_create_table_from_external_file] + + +Your ``dags/create_table.sql`` should look like this: + +.. code-block:: sql + + -- create Users table + CREATE TABLE Users, FALLBACK ( + username varchar(50), + description varchar(256) + ); + + +Inserting data into a Teradata database table +--------------------------------------------- +We can then create a TeradataOperator task that populate the ``Users`` table. + +.. exampleinclude:: /../../tests/system/providers/teradata/example_teradata.py + :language: python + :start-after: [START teradata_operator_howto_guide_populate_table] + :end-before: [END teradata_operator_howto_guide_populate_table] + + +Fetching records from your Teradata database table +-------------------------------------------------- + +Fetching records from your Teradata database table can be as simple as: + +.. exampleinclude:: /../../tests/system/providers/teradata/example_teradata.py + :language: python + :start-after: [START teradata_operator_howto_guide_get_all_countries] + :end-before: [END teradata_operator_howto_guide_get_all_countries] + + +Passing Parameters into TeradataOperator +---------------------------------------- + +TeradataOperator provides ``parameters`` attribute which makes it possible to dynamically inject values into your +SQL requests during runtime. + +To find the countries in Asian continent: + +.. exampleinclude:: /../../tests/system/providers/teradata/example_teradata.py + :language: python + :start-after: [START teradata_operator_howto_guide_params_passing_get_query] + :end-before: [END teradata_operator_howto_guide_params_passing_get_query] + + +Dropping a Teradata database table +-------------------------------------------------- + +We can then create a TeradataOperator task that drops the ``Users`` table. + +.. exampleinclude:: /../../tests/system/providers/teradata/example_teradata.py + :language: python + :start-after: [START teradata_operator_howto_guide_drop_users_table] + :end-before: [END teradata_operator_howto_guide_drop_users_table] + +The complete Teradata Operator DAG +---------------------------------- + +When we put everything together, our DAG should look like this: .. exampleinclude:: /../../tests/system/providers/teradata/example_teradata.py :language: python diff --git a/docs/apache-airflow-providers-teradata/operators/teradata_to_teradata.rst b/docs/apache-airflow-providers-teradata/operators/teradata_to_teradata.rst index 20c76c2d9bc825..6c551427f0737c 100644 --- a/docs/apache-airflow-providers-teradata/operators/teradata_to_teradata.rst +++ b/docs/apache-airflow-providers-teradata/operators/teradata_to_teradata.rst @@ -34,5 +34,15 @@ An example usage of the TeradataToTeradataOperator is as follows: .. exampleinclude:: /../../tests/system/providers/teradata/example_teradata_to_teradata_transfer.py :language: python - :start-after: [START teradata_to_teradata_transfer_operator_howto_guide] - :end-before: [END teradata_to_teradata_transfer_operator_howto_guide] + :start-after: [START teradata_to_teradata_transfer_operator_howto_guide_transfer_data] + :end-before: [END teradata_to_teradata_transfer_operator_howto_guide_transfer_data] + +The complete TeradataToTeradata Transfer Operator DAG +----------------------------------------------------- + +When we put everything together, our DAG should look like this: + +.. exampleinclude:: /../../tests/system/providers/teradata/example_teradata.py + :language: python + :start-after: [START teradata_operator_howto_guide] + :end-before: [END teradata_operator_howto_guide] diff --git a/docs/apache-airflow-providers-teradata/redirects.txt b/docs/apache-airflow-providers-teradata/redirects.txt deleted file mode 100644 index 74ee3f22fa30c8..00000000000000 --- a/docs/apache-airflow-providers-teradata/redirects.txt +++ /dev/null @@ -1 +0,0 @@ -connections/index.rst connections/teradata.rst diff --git a/pyproject.toml b/pyproject.toml index f60487a40ebc05..12498570879055 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -926,7 +926,7 @@ teradata = [ # source: airflow/providers/teradata/provider.yaml "teradatasql>=17.20.0.28", "teradatasqlalchemy>=17.20.0.0", ] -trino = [ +trino = [ # source: airflow/providers/trino/provider.yaml "apache-airflow[common_sql]", "pandas>=1.2.5", "trino>=0.318.0", diff --git a/tests/providers/teradata/hooks/test_teradata.py b/tests/providers/teradata/hooks/test_teradata.py index 8f7725cbdaa8c2..d47e987f41f22e 100644 --- a/tests/providers/teradata/hooks/test_teradata.py +++ b/tests/providers/teradata/hooks/test_teradata.py @@ -24,11 +24,9 @@ import pytest from airflow.models import Connection - from airflow.providers.teradata.hooks.teradata import TeradataHook - class TestTeradataHook: def setup_method(self): self.connection = Connection( diff --git a/tests/providers/teradata/operators/test_teradata.py b/tests/providers/teradata/operators/test_teradata.py index ca9e9fb44cf866..67f27900172ba6 100644 --- a/tests/providers/teradata/operators/test_teradata.py +++ b/tests/providers/teradata/operators/test_teradata.py @@ -20,15 +20,12 @@ from unittest import mock from unittest.mock import MagicMock, Mock +from airflow.exceptions import AirflowException from airflow.models.dag import DAG from airflow.providers.common.sql.hooks.sql import fetch_all_handler -from airflow.utils import timezone - from airflow.providers.teradata.hooks.teradata import TeradataHook from airflow.providers.teradata.operators.teradata import TeradataOperator - - -from airflow.exceptions import AirflowException +from airflow.utils import timezone DEFAULT_DATE = timezone.datetime(2015, 1, 1) TEST_DAG_ID = "unit_test_dag" @@ -79,10 +76,7 @@ def test_execute(self, mock_get_db_hook): task_id = "test_task_id" operator = TeradataOperator( - sql=sql, - conn_id=teradata_conn_id, - parameters=parameters, - task_id=task_id, + sql=sql, conn_id=teradata_conn_id, parameters=parameters, task_id=task_id, dag=self.dag ) operator.execute(context=context) mock_get_db_hook.return_value.run.assert_called_once_with( @@ -107,10 +101,7 @@ def test_teradata_operator_test_multi(self, mock_get_db_hook): task_id = "test_task_id" operator = TeradataOperator( - sql=sql, - conn_id=teradata_conn_id, - parameters=parameters, - task_id=task_id, + sql=sql, conn_id=teradata_conn_id, parameters=parameters, task_id=task_id, dag=self.dag ) operator.execute(context=context) mock_get_db_hook.return_value.run.assert_called_once_with( diff --git a/tests/system/providers/teradata/example_teradata.py b/tests/system/providers/teradata/example_teradata.py index a7fd039c1d95b7..d13bec5ef34534 100644 --- a/tests/system/providers/teradata/example_teradata.py +++ b/tests/system/providers/teradata/example_teradata.py @@ -35,10 +35,6 @@ # [START teradata_operator_howto_guide] -# create_table_teradata, insert_teradata_task, create_table_teradata_from_external_file, populate_user_table -# get_all_countries, get_all_description, get_countries_from_continent, drop_table_teradata_task, drop_users_table_teradata_task -# are examples of tasks created by instantiating the Teradata Operator - ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID") DAG_ID = "example_teradata" @@ -49,9 +45,9 @@ catchup=False, default_args={"conn_id": "teradata_default"}, ) as dag: - # [START teradata_operator_howto_guide_create_country_table] - create_country_table = TeradataOperator( - task_id="create_country_table", + # [START teradata_operator_howto_guide_create_table] + create_table = TeradataOperator( + task_id="create_table", sql=r""" CREATE TABLE Country ( country_id INTEGER, @@ -60,55 +56,116 @@ ); """, ) - # [END teradata_operator_howto_guide_create_country_table] - # [START teradata_operator_howto_guide_populate_country_table] - populate_country_table = TeradataOperator( - task_id="populate_country_table", - sql=r""" - INSERT INTO Country VALUES ( 1, 'India', 'Asia'); - INSERT INTO Country VALUES ( 2, 'Germany', 'Europe'); - INSERT INTO Country VALUES ( 3, 'Argentina', 'South America'); - INSERT INTO Country VALUES ( 4, 'Ghana', 'Africa'); - """, - ) - # [END teradata_operator_howto_guide_populate_country_table] - # [START teradata_operator_howto_guide_create_users_table_from_external_file] - create_users_table_from_external_file = TeradataOperator( - task_id="create_users_table_from_external_file", + # [END teradata_operator_howto_guide_create_table] + + # [START teradata_operator_howto_guide_create_table_from_external_file] + create_table_from_external_file = TeradataOperator( + task_id="create_table_from_external_file", sql="create_table.sql", dag=dag, ) - # [END teradata_operator_howto_guide_create_users_table_from_external_file] + # [END teradata_operator_howto_guide_create_table_from_external_file] + # [START teradata_operator_howto_guide_populate_table] + populate_table = TeradataOperator( + task_id="populate_table", + sql=r""" + INSERT INTO Users (username, description) + VALUES ( 'Danny', 'Musician'); + INSERT INTO Users (username, description) + VALUES ( 'Simone', 'Chef'); + INSERT INTO Users (username, description) + VALUES ( 'Lily', 'Florist'); + INSERT INTO Users (username, description) + VALUES ( 'Tim', 'Pet shop owner'); + """, + ) + # [END teradata_operator_howto_guide_populate_table] # [START teradata_operator_howto_guide_get_all_countries] get_all_countries = TeradataOperator( task_id="get_all_countries", - sql=r"""SELECT * FROM Country;""", + sql=r""" + SELECT * FROM Country; + """, ) # [END teradata_operator_howto_guide_get_all_countries] # [START teradata_operator_howto_guide_params_passing_get_query] get_countries_from_continent = TeradataOperator( task_id="get_countries_from_continent", - sql=r"""SELECT * FROM Country where {{ params.column }}='{{ params.value }}';""", + sql=r""" + SELECT * FROM Country WHERE {{ params.column }}='{{ params.value }}'; + """, params={"column": "continent", "value": "Asia"}, ) # [END teradata_operator_howto_guide_params_passing_get_query] # [START teradata_operator_howto_guide_drop_country_table] drop_country_table = TeradataOperator( - task_id="drop_country_table", sql=r"""DROP TABLE Country;""", dag=dag + task_id="drop_country_table", + sql=r""" + DROP TABLE Country; + """, + dag=dag ) # [END teradata_operator_howto_guide_drop_country_table] # [START teradata_operator_howto_guide_drop_users_table] - drop_users_table = TeradataOperator(task_id="drop_users_table", sql=r"""DROP TABLE Users;""", dag=dag) + drop_users_table = TeradataOperator( + task_id="drop_users_table", + sql=r""" + DROP TABLE Users; + """, + dag=dag) # [END teradata_operator_howto_guide_drop_users_table] + # [START teradata_operator_howto_guide_create_schema] + create_schema = TeradataOperator( + task_id="create_schema", + sql=r""" + CREATE DATABASE airflow_temp AS PERM=10e6; + """, + ) + # [END teradata_operator_howto_guide_create_schema] + # [START teradata_operator_howto_guide_create_table_with_schema] + create_table_with_schema = TeradataOperator( + task_id="create_table_with_schema", + sql=r""" + CREATE TABLE schema_table ( + country_id INTEGER, + name CHAR(25), + continent CHAR(25) + ); + """, + schema="airflow_temp", + ) + # [END teradata_operator_howto_guide_create_table_with_schema] + # [START teradata_operator_howto_guide_drop_schema_table] + drop_schema_table = TeradataOperator( + task_id="drop_schema_table", + sql=r""" + DROP TABLE schema_table; + """, + dag=dag, + schema="airflow_temp", + ) + # [END teradata_operator_howto_guide_drop_schema_table] + # [START teradata_operator_howto_guide_drop_schema] + drop_schema = TeradataOperator( + task_id="drop_schema", + sql=r""" + DROP DATABASE airflow_temp; + """, + dag=dag) + # [END teradata_operator_howto_guide_drop_schema] ( - create_country_table - >> populate_country_table - >> create_users_table_from_external_file + create_table + >> create_table_from_external_file + >> populate_table >> get_all_countries >> get_countries_from_continent >> drop_country_table >> drop_users_table + >> create_schema + >> create_table_with_schema + >> drop_schema_table + >> drop_schema ) # [END teradata_operator_howto_guide] diff --git a/tests/system/providers/teradata/example_teradata_to_teradata_transfer.py b/tests/system/providers/teradata/example_teradata_to_teradata_transfer.py index 8727715f8a6495..8519691154cda8 100644 --- a/tests/system/providers/teradata/example_teradata_to_teradata_transfer.py +++ b/tests/system/providers/teradata/example_teradata_to_teradata_transfer.py @@ -41,11 +41,6 @@ # [START teradata_to_teradata_transfer_operator_howto_guide] - -# create_src_table, create_dest_table, insert_data_src, read_data_src, read_data_dest, drop_src_table -# and drop_dest_table are examples of tasks created by instantiating the Teradata Operator and transfer_data -# is the example of task created by instantiating the TeradataToTeradata Transfer Operator. - ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID") DAG_ID = "example_teradata_to_teradata_transfer_operator" CONN_ID = "teradata_default" From 06500bb96723ea34060a32cd1b78f74ac800f4df Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Mon, 5 Feb 2024 09:42:29 -0800 Subject: [PATCH 12/21] static check format applied --- tests/system/providers/teradata/example_teradata.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/system/providers/teradata/example_teradata.py b/tests/system/providers/teradata/example_teradata.py index d13bec5ef34534..d02458454f992d 100644 --- a/tests/system/providers/teradata/example_teradata.py +++ b/tests/system/providers/teradata/example_teradata.py @@ -103,7 +103,7 @@ sql=r""" DROP TABLE Country; """, - dag=dag + dag=dag, ) # [END teradata_operator_howto_guide_drop_country_table] # [START teradata_operator_howto_guide_drop_users_table] @@ -112,7 +112,8 @@ sql=r""" DROP TABLE Users; """, - dag=dag) + dag=dag, + ) # [END teradata_operator_howto_guide_drop_users_table] # [START teradata_operator_howto_guide_create_schema] create_schema = TeradataOperator( @@ -151,7 +152,8 @@ sql=r""" DROP DATABASE airflow_temp; """, - dag=dag) + dag=dag, + ) # [END teradata_operator_howto_guide_drop_schema] ( From bf13d02dbc1011a943dbd174269733ff4a566353 Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Tue, 6 Feb 2024 13:29:57 +0530 Subject: [PATCH 13/21] Review comments addressed --- airflow/providers/teradata/hooks/teradata.py | 20 +++++-------------- .../operators/teradata.rst | 2 +- .../teradata/operators/test_teradata.py | 18 ++--------------- 3 files changed, 8 insertions(+), 32 deletions(-) diff --git a/airflow/providers/teradata/hooks/teradata.py b/airflow/providers/teradata/hooks/teradata.py index fc3f3d521a03eb..5f5d1ed8fbd5c4 100644 --- a/airflow/providers/teradata/hooks/teradata.py +++ b/airflow/providers/teradata/hooks/teradata.py @@ -15,7 +15,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -"""A Airflow Hook for interacting with Teradata SQL Server.""" +"""An Airflow Hook for interacting with Teradata SQL Server.""" from __future__ import annotations from typing import TYPE_CHECKING, Any @@ -38,21 +38,15 @@ class TeradataHook(DbApiHook): Teradata DB Server URL, username, password and database name are fetched from the predefined connection config connection_id. It raises an airflow error if the given connection id doesn't exist. + You can also specify ssl parameters in the extra field of your connection + as ``{"sslmode": "require", "sslcert": "/path/to/cert.pem", etc}``. + .. seealso:: - :ref:`Teradata API connection ` :param args: passed to DbApiHook :param database: The Teradata database to connect to. :param kwargs: passed to DbApiHook - - - Usage Help: - - >>> tdh = TeradataHook() - >>> sql = "SELECT top 1 _airbyte_ab_id from airbyte_td._airbyte_raw_Sales;" - >>> tdh.get_records(sql) - [[61ad1d63-3efd-4da4-9904-a4489cc3a520]] - """ # Override to provide the connection name. @@ -89,11 +83,7 @@ def get_conn(self) -> TeradataConnection: Establishes connection to a Teradata SQL database using config corresponding to teradata_conn_id. - .. note:: By default it connects to the database via the teradatasql library. - But you can also choose the mysql-connector-python library which lets you connect through ssl - without any further ssl parameters required. - - :return: a mysql connection object + :return: a Teradata connection object """ teradata_conn_config: dict = self._get_conn_config_teradatasql() teradata_conn = teradatasql.connect(**teradata_conn_config) diff --git a/docs/apache-airflow-providers-teradata/operators/teradata.rst b/docs/apache-airflow-providers-teradata/operators/teradata.rst index f0e0a299b120d7..2e98341a4cc121 100644 --- a/docs/apache-airflow-providers-teradata/operators/teradata.rst +++ b/docs/apache-airflow-providers-teradata/operators/teradata.rst @@ -38,7 +38,7 @@ An example usage of the TeradataOperator is as follows: :start-after: [START teradata_operator_howto_guide_create_table] :end-before: [END teradata_operator_howto_guide_create_table] -You can also use an external file to execute the SQL commands. Script folder must be at the same level as DAG.py file. +You can also use an external file to execute the SQL commands. External file must be at the same level as DAG.py file. This way you can easily maintain the SQL queries separated from the code. .. exampleinclude:: /../../tests/system/providers/teradata/example_teradata.py diff --git a/tests/providers/teradata/operators/test_teradata.py b/tests/providers/teradata/operators/test_teradata.py index 67f27900172ba6..3d491a66758b40 100644 --- a/tests/providers/teradata/operators/test_teradata.py +++ b/tests/providers/teradata/operators/test_teradata.py @@ -21,22 +21,12 @@ from unittest.mock import MagicMock, Mock from airflow.exceptions import AirflowException -from airflow.models.dag import DAG from airflow.providers.common.sql.hooks.sql import fetch_all_handler from airflow.providers.teradata.hooks.teradata import TeradataHook from airflow.providers.teradata.operators.teradata import TeradataOperator -from airflow.utils import timezone - -DEFAULT_DATE = timezone.datetime(2015, 1, 1) -TEST_DAG_ID = "unit_test_dag" class TestTeradataOperator: - def setup_method(self): - args = {"owner": "airflow", "start_date": DEFAULT_DATE} - dag = DAG(TEST_DAG_ID, default_args=args) - self.dag = dag - @mock.patch("airflow.providers.common.sql.operators.sql.SQLExecuteQueryOperator.get_db_hook") def test_get_hook_from_conn(self, mock_get_db_hook): """ @@ -75,9 +65,7 @@ def test_execute(self, mock_get_db_hook): context = "test_context" task_id = "test_task_id" - operator = TeradataOperator( - sql=sql, conn_id=teradata_conn_id, parameters=parameters, task_id=task_id, dag=self.dag - ) + operator = TeradataOperator(sql=sql, conn_id=teradata_conn_id, parameters=parameters, task_id=task_id) operator.execute(context=context) mock_get_db_hook.return_value.run.assert_called_once_with( sql=sql, @@ -100,9 +88,7 @@ def test_teradata_operator_test_multi(self, mock_get_db_hook): context = "test_context" task_id = "test_task_id" - operator = TeradataOperator( - sql=sql, conn_id=teradata_conn_id, parameters=parameters, task_id=task_id, dag=self.dag - ) + operator = TeradataOperator(sql=sql, conn_id=teradata_conn_id, parameters=parameters, task_id=task_id) operator.execute(context=context) mock_get_db_hook.return_value.run.assert_called_once_with( sql=sql, From d1733e35c0820d8b74c8d5995cfc63633eaf5ef8 Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Fri, 9 Feb 2024 03:21:52 -0800 Subject: [PATCH 14/21] static checks issues resolved --- INSTALL | 4 ++-- contributing-docs/12_airflow_dependencies_and_extras.rst | 4 ++-- dev/breeze/doc/images/output_build-docs.svg | 2 +- dev/breeze/doc/images/output_build-docs.txt | 2 +- .../images/output_release-management_add-back-references.svg | 2 +- .../images/output_release-management_add-back-references.txt | 2 +- ...ut_release-management_generate-issue-content-providers.svg | 2 +- ...ut_release-management_generate-issue-content-providers.txt | 2 +- ...tput_release-management_prepare-provider-documentation.svg | 2 +- ...tput_release-management_prepare-provider-documentation.txt | 2 +- .../output_release-management_prepare-provider-packages.svg | 2 +- .../output_release-management_prepare-provider-packages.txt | 2 +- .../doc/images/output_release-management_publish-docs.svg | 2 +- .../doc/images/output_release-management_publish-docs.txt | 2 +- .../images/output_sbom_generate-providers-requirements.svg | 2 +- .../images/output_sbom_generate-providers-requirements.txt | 2 +- 16 files changed, 18 insertions(+), 18 deletions(-) diff --git a/INSTALL b/INSTALL index c27cc3f3ef34db..a54f98114aee52 100644 --- a/INSTALL +++ b/INSTALL @@ -255,8 +255,8 @@ microsoft-mssql, microsoft-psrp, microsoft-winrm, mongo, mssql, mysql, neo4j, od openfaas, openlineage, opensearch, opsgenie, oracle, otel, pagerduty, pandas, papermill, password, pgvector, pinecone, pinot, postgres, presto, qdrant, rabbitmq, redis, s3, s3fs, salesforce, samba, saml, segment, sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, -statsd, tableau, tabular, telegram, teradata, trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex, -zendesk +statsd, tableau, tabular, telegram, teradata, trino, vertica, virtualenv, weaviate, webhdfs, winrm, +yandex, zendesk # END REGULAR EXTRAS HERE diff --git a/contributing-docs/12_airflow_dependencies_and_extras.rst b/contributing-docs/12_airflow_dependencies_and_extras.rst index 0105a8a6635740..46bf125b8cef07 100644 --- a/contributing-docs/12_airflow_dependencies_and_extras.rst +++ b/contributing-docs/12_airflow_dependencies_and_extras.rst @@ -211,8 +211,8 @@ microsoft-mssql, microsoft-psrp, microsoft-winrm, mongo, mssql, mysql, neo4j, od openfaas, openlineage, opensearch, opsgenie, oracle, otel, pagerduty, pandas, papermill, password, pgvector, pinecone, pinot, postgres, presto, qdrant, rabbitmq, redis, s3, s3fs, salesforce, samba, saml, segment, sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, -statsd, tableau, tabular, telegram, teradata, trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex, -zendesk +statsd, tableau, tabular, telegram, teradata, trino, vertica, virtualenv, weaviate, webhdfs, winrm, +yandex, zendesk .. END REGULAR EXTRAS HERE diff --git a/dev/breeze/doc/images/output_build-docs.svg b/dev/breeze/doc/images/output_build-docs.svg index bab81e920a3562..a7b8044ea2808a 100644 --- a/dev/breeze/doc/images/output_build-docs.svg +++ b/dev/breeze/doc/images/output_build-docs.svg @@ -184,7 +184,7 @@ microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage | opensearch |       opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | qdrant | redis | salesforce |    samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular |        -telegram | trino | vertica | weaviate | yandex | zendesk]...                                                           +telegram | teradata | trino | vertica | weaviate | yandex | zendesk]...                                                Build documents. diff --git a/dev/breeze/doc/images/output_build-docs.txt b/dev/breeze/doc/images/output_build-docs.txt index 1a93849cc2fbe8..6433309e2c517e 100644 --- a/dev/breeze/doc/images/output_build-docs.txt +++ b/dev/breeze/doc/images/output_build-docs.txt @@ -1 +1 @@ -8f5e4f90611d600cbd02ae9e79d60df2 +a074c21108ac2bffc10bf20183a19d7f diff --git a/dev/breeze/doc/images/output_release-management_add-back-references.svg b/dev/breeze/doc/images/output_release-management_add-back-references.svg index 7492e1d03f88dd..48a3be2224e818 100644 --- a/dev/breeze/doc/images/output_release-management_add-back-references.svg +++ b/dev/breeze/doc/images/output_release-management_add-back-references.svg @@ -146,7 +146,7 @@ microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage | opensearch |       opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | qdrant | redis | salesforce |    samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular |        -telegram | trino | vertica | weaviate | yandex | zendesk]...                                                           +telegram | teradata | trino | vertica | weaviate | yandex | zendesk]...                                                Command to add back references for documentation to make it backward compatible. diff --git a/dev/breeze/doc/images/output_release-management_add-back-references.txt b/dev/breeze/doc/images/output_release-management_add-back-references.txt index bd8679b924e60e..ae51a4106f10af 100644 --- a/dev/breeze/doc/images/output_release-management_add-back-references.txt +++ b/dev/breeze/doc/images/output_release-management_add-back-references.txt @@ -1 +1 @@ -af0db4105f4aec228083f240d550bda3 +6cccd29cb919026e925f9c54882c4900 diff --git a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg index 0646eb58711771..9e98a5fafeb644 100644 --- a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg +++ b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg @@ -146,7 +146,7 @@ microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage | opensearch |     opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | qdrant | redis | salesforce |    samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular |        -telegram | trino | vertica | weaviate | yandex | zendesk]...                                                           +telegram | teradata | trino | vertica | weaviate | yandex | zendesk]...                                                Generates content for issue to test the release. diff --git a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt index 36d7be3c996996..5f5f8e80244b10 100644 --- a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt +++ b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt @@ -1 +1 @@ -97f88e5ddbf7bd0f8de4fb734c8a2386 +1f93a3551c892c6be934a31c212d2457 diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg index 34893298dbeec1..91f478d035147c 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg @@ -182,7 +182,7 @@ microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage | opensearch |     opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | qdrant | redis | salesforce |    samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular |        -telegram | trino | vertica | weaviate | yandex | zendesk]...                                                           +telegram | teradata | trino | vertica | weaviate | yandex | zendesk]...                                                Prepare CHANGELOG, README and COMMITS information for providers. diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt index 83b69b7ea721ef..d17f57b76d48b4 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt @@ -1 +1 @@ -663614748d86a8e2e8df08417e9b9307 +97ac609c3c46f5d85255f4f66c9012a0 diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-packages.svg b/dev/breeze/doc/images/output_release-management_prepare-provider-packages.svg index 0104899650f573..9a021aff840613 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-packages.svg +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-packages.svg @@ -167,7 +167,7 @@ microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage | opensearch |     opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | qdrant | redis | salesforce |    samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular |        -telegram | trino | vertica | weaviate | yandex | zendesk]...                                                           +telegram | teradata | trino | vertica | weaviate | yandex | zendesk]...                                                Prepare sdist/whl packages of Airflow Providers. diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-packages.txt b/dev/breeze/doc/images/output_release-management_prepare-provider-packages.txt index a65dfccd0b6fd1..879c659a0dcf9f 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-packages.txt +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-packages.txt @@ -1 +1 @@ -c233e9c9a308ce97422dfb26a4125ada +8cbc0d648fa0dfcdfeec881fc6bbdfc7 diff --git a/dev/breeze/doc/images/output_release-management_publish-docs.svg b/dev/breeze/doc/images/output_release-management_publish-docs.svg index 0f8a6216c49429..02ec6e6e87bf73 100644 --- a/dev/breeze/doc/images/output_release-management_publish-docs.svg +++ b/dev/breeze/doc/images/output_release-management_publish-docs.svg @@ -192,7 +192,7 @@ microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage | opensearch |       opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | qdrant | redis | salesforce |    samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular |        -telegram | trino | vertica | weaviate | yandex | zendesk]...                                                           +telegram | teradata | trino | vertica | weaviate | yandex | zendesk]...                                                Command to publish generated documentation to airflow-site diff --git a/dev/breeze/doc/images/output_release-management_publish-docs.txt b/dev/breeze/doc/images/output_release-management_publish-docs.txt index d3a965a4bbc211..75e8506095276f 100644 --- a/dev/breeze/doc/images/output_release-management_publish-docs.txt +++ b/dev/breeze/doc/images/output_release-management_publish-docs.txt @@ -1 +1 @@ -30bde47bb7c648532bcadd4c53ff3d1e +7988adc4d04440c2df05551418e8ffe7 diff --git a/dev/breeze/doc/images/output_sbom_generate-providers-requirements.svg b/dev/breeze/doc/images/output_sbom_generate-providers-requirements.svg index 6505980e871242..6a23a0af83f1ed 100644 --- a/dev/breeze/doc/images/output_sbom_generate-providers-requirements.svg +++ b/dev/breeze/doc/images/output_sbom_generate-providers-requirements.svg @@ -193,7 +193,7 @@ │| neo4j | odbc | openai | openfaas | openlineage | opensearch | opsgenie | oracle | pagerduty â”‚ │| papermill | pgvector | pinecone | postgres | presto | qdrant | redis | salesforce | samba | â”‚ │segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | â”‚ -│tabular | telegram | trino | vertica | weaviate | yandex | zendesk)                           â”‚ +│tabular | telegram | teradata | trino | vertica | weaviate | yandex | zendesk)                â”‚ │--provider-versionProvider version to generate the requirements for i.e `2.1.0`. `latest` is also a supported   â”‚ │value to account for the most recent version of the provider                                  â”‚ │(TEXT)                                                                                        â”‚ diff --git a/dev/breeze/doc/images/output_sbom_generate-providers-requirements.txt b/dev/breeze/doc/images/output_sbom_generate-providers-requirements.txt index 6eab2584f87cf4..aa69d18e37c91e 100644 --- a/dev/breeze/doc/images/output_sbom_generate-providers-requirements.txt +++ b/dev/breeze/doc/images/output_sbom_generate-providers-requirements.txt @@ -1 +1 @@ -75d28480ee1900ffd878862190585efc +35e7f8e2d5294ae6522c9f6719a292e1 From db357831919d824eebb29f15b6f92400142baad1 Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Sat, 10 Feb 2024 00:13:27 +0530 Subject: [PATCH 15/21] Squashed commit of the following: commit 58279b8573b06c76644fa42f736a616ca01baeb9 Author: Satish Ch Date: Sat Feb 10 00:12:42 2024 +0530 Update teradata.py commit ed75e3e7cae8d10b06bf73181b6f4066df95b6b5 Author: Satish Ch Date: Sat Feb 10 00:09:24 2024 +0530 D401 support in Teradata Provider commit f56bede5fb8866dff4720cd4d39fe9e39d6249e8 Merge: e859a1d59a 00ed46769e Author: Satish Ch Date: Fri Feb 9 23:58:15 2024 +0530 Merge remote-tracking branch 'upstream/main' into pr_teradata_release_1.0.0 commit 00ed46769eaea24251fc4726a46df1f54f27c4bd Author: Vincent <97131062+vincbeck@users.noreply.github.com> Date: Fri Feb 9 13:13:36 2024 -0500 D401 support in fab provider (#37283) commit 48bfb1a970f5b47ba1b385ad809b8324923ddf3e Author: Niko Oliveira Date: Fri Feb 9 08:43:32 2024 -0800 Merge all ECS executor configs following recursive python dict update (#37137) Also document the behaviour and interaction between exec_config and run_task_kwargs config commit 8317ed93a58900d922ab4ca8da02ed1c6050252c Author: Amogh Desai Date: Fri Feb 9 20:17:38 2024 +0530 Updating the README and visuals for breeze build-docs (#37276) commit 17945fc5edcec619ccde0fbab0d6fb8e0eb206cd Author: Kalyan Date: Fri Feb 9 20:16:33 2024 +0530 D401 fixes in Pinecone provider (#37270) commit ab9e2e166eb363d8316b24b8548d401faa7d517b Author: Kalyan Date: Fri Feb 9 20:15:31 2024 +0530 fix: D401 lint issues in airflow core (#37274) commit 7835fd2659335d3acf830ce2e70dc19bfc5b2a84 Author: Jarek Potiuk Date: Fri Feb 9 14:59:33 2024 +0100 The fix-ownership command missed --rm flag and left dangling containers (#37277) Fixes: #37269 commit e859a1d59ae913cae51548d57806d7f17da43369 Author: Satish Ch Date: Fri Feb 9 19:21:28 2024 +0530 Update teradata.py commit d1c08e1137924cdd449efa4dd197cef158634ed0 Author: Satish Ch Date: Fri Feb 9 19:18:37 2024 +0530 Update teradata.py commit e5ac0ecc592a5fb470aa50afad1240af90d5a8b3 Author: Satish Ch Date: Fri Feb 9 05:41:42 2024 -0800 static check issue is fixed commit ce490f7579c4fea637d46678f27f05a762cb1070 Author: Satish Ch Date: Fri Feb 9 19:10:36 2024 +0530 static format issue fixed commit f9498c532558ea93a4c40a56d2b7ce19f8be52a6 Author: Satish Ch Date: Fri Feb 9 05:36:36 2024 -0800 static check issue is fixed commit 77bddae56613297229b8c44f3df6acf1ad58a124 Author: Satish Ch Date: Fri Feb 9 18:23:18 2024 +0530 common sql unit test failure fixed commit 9f4f208b5da38bc2e82db682c636ec4fcf7ad617 Author: Aleksey Kirilishin <54231417+avkirilishin@users.noreply.github.com> Date: Fri Feb 9 15:53:04 2024 +0300 Fix the bug that affected the DAG end date. (#36144) commit 0f8dfebdd6cd30e604d2180998f976d0c7b62277 Author: Sudipto Baral Date: Fri Feb 9 06:54:23 2024 -0500 fix: update hyperlink to the new documentation section for local virtualenv setup. (#37272) Signed-off-by: sudipto baral --- airflow/api/common/mark_tasks.py | 5 - airflow/auth/managers/utils/fab.py | 4 +- airflow/decorators/bash.py | 2 +- airflow/executors/debug_executor.py | 2 +- airflow/models/baseoperator.py | 2 +- airflow/models/dagrun.py | 70 +++++- airflow/models/taskinstance.py | 12 +- airflow/operators/python.py | 4 +- airflow/plugins_manager.py | 6 +- .../amazon/aws/executors/ecs/ecs_executor.py | 22 +- .../fab/auth_manager/decorators/auth.py | 2 +- .../fab/auth_manager/fab_auth_manager.py | 2 +- .../auth_manager/security_manager/override.py | 52 ++--- airflow/providers/pinecone/hooks/pinecone.py | 14 +- airflow/providers/teradata/hooks/teradata.py | 10 +- airflow/providers_manager.py | 26 +-- airflow/serialization/serde.py | 6 +- airflow/utils/file.py | 2 +- airflow/utils/log/task_context_logger.py | 2 +- airflow/utils/sqlalchemy.py | 2 + airflow/www/auth.py | 4 +- airflow/www/blueprints.py | 2 +- airflow/www/views.py | 2 +- .../06_development_environments.rst | 2 +- .../commands/developer_commands.py | 6 +- .../utils/docker_command_utils.py | 1 + dev/breeze/tests/test_selective_checks.py | 2 +- docs/README.rst | 13 +- .../executors/ecs-executor.rst | 8 +- pyproject.toml | 20 -- .../endpoints/test_dag_run_endpoint.py | 4 +- .../client/test_local_client.py | 4 +- .../common/test_mark_tasks.py | 49 ++-- tests/models/test_cleartasks.py | 5 +- .../aws/executors/ecs/test_ecs_executor.py | 218 ++++++++++++++++++ 35 files changed, 442 insertions(+), 145 deletions(-) diff --git a/airflow/api/common/mark_tasks.py b/airflow/api/common/mark_tasks.py index 3cc6dfdfd715b6..a175a61e207ea6 100644 --- a/airflow/api/common/mark_tasks.py +++ b/airflow/api/common/mark_tasks.py @@ -366,11 +366,6 @@ def _set_dag_run_state(dag_id: str, run_id: str, state: DagRunState, session: SA select(DagRun).where(DagRun.dag_id == dag_id, DagRun.run_id == run_id) ).scalar_one() dag_run.state = state - if state == DagRunState.RUNNING: - dag_run.start_date = timezone.utcnow() - dag_run.end_date = None - else: - dag_run.end_date = timezone.utcnow() session.merge(dag_run) diff --git a/airflow/auth/managers/utils/fab.py b/airflow/auth/managers/utils/fab.py index 316e5ecff1658d..22b572e07f5053 100644 --- a/airflow/auth/managers/utils/fab.py +++ b/airflow/auth/managers/utils/fab.py @@ -40,12 +40,12 @@ def get_fab_action_from_method_map(): - """Returns the map associating a method to a FAB action.""" + """Return the map associating a method to a FAB action.""" return _MAP_METHOD_NAME_TO_FAB_ACTION_NAME def get_method_from_fab_action_map(): - """Returns the map associating a FAB action to a method.""" + """Return the map associating a FAB action to a method.""" return { **{v: k for k, v in _MAP_METHOD_NAME_TO_FAB_ACTION_NAME.items()}, ACTION_CAN_ACCESS_MENU: "GET", diff --git a/airflow/decorators/bash.py b/airflow/decorators/bash.py index 70011c30790346..36fc646370ee73 100644 --- a/airflow/decorators/bash.py +++ b/airflow/decorators/bash.py @@ -84,7 +84,7 @@ def bash_task( python_callable: Callable | None = None, **kwargs, ) -> TaskDecorator: - """Wraps a function into a BashOperator. + """Wrap a function into a BashOperator. Accepts kwargs for operator kwargs. Can be reused in a single DAG. This function is only used only used during type checking or auto-completion. diff --git a/airflow/executors/debug_executor.py b/airflow/executors/debug_executor.py index 750b0ba20b033e..9b376cdb010228 100644 --- a/airflow/executors/debug_executor.py +++ b/airflow/executors/debug_executor.py @@ -61,7 +61,7 @@ def __init__(self): self.fail_fast = conf.getboolean("debug", "fail_fast") def execute_async(self, *args, **kwargs) -> None: - """The method is replaced by custom trigger_task implementation.""" + """Replace the method with a custom trigger_task implementation.""" def sync(self) -> None: task_succeeded = True diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py index ce55b24350b494..e2406776d86c0d 100644 --- a/airflow/models/baseoperator.py +++ b/airflow/models/baseoperator.py @@ -1602,7 +1602,7 @@ def defer( raise TaskDeferred(trigger=trigger, method_name=method_name, kwargs=kwargs, timeout=timeout) def resume_execution(self, next_method: str, next_kwargs: dict[str, Any] | None, context: Context): - """This method is called when a deferred task is resumed.""" + """Call this method when a deferred task is resumed.""" # __fail__ is a special signal value for next_method that indicates # this task was scheduled specifically to fail. if next_method == "__fail__": diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 6b4fdbd0cf39c2..f9126dd6313bf5 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -269,11 +269,75 @@ def get_state(self): return self._state def set_state(self, state: DagRunState) -> None: + """Change the state of the DagRan. + + Changes to attributes are implemented in accordance with the following table + (rows represent old states, columns represent new states): + + .. list-table:: State transition matrix + :header-rows: 1 + :stub-columns: 1 + + * - + - QUEUED + - RUNNING + - SUCCESS + - FAILED + * - None + - queued_at = timezone.utcnow() + - if empty: start_date = timezone.utcnow() + end_date = None + - end_date = timezone.utcnow() + - end_date = timezone.utcnow() + * - QUEUED + - queued_at = timezone.utcnow() + - if empty: start_date = timezone.utcnow() + end_date = None + - end_date = timezone.utcnow() + - end_date = timezone.utcnow() + * - RUNNING + - queued_at = timezone.utcnow() + start_date = None + end_date = None + - + - end_date = timezone.utcnow() + - end_date = timezone.utcnow() + * - SUCCESS + - queued_at = timezone.utcnow() + start_date = None + end_date = None + - start_date = timezone.utcnow() + end_date = None + - + - + * - FAILED + - queued_at = timezone.utcnow() + start_date = None + end_date = None + - start_date = timezone.utcnow() + end_date = None + - + - + + """ if state not in State.dag_states: raise ValueError(f"invalid DagRun state: {state}") if self._state != state: + if state == DagRunState.QUEUED: + self.queued_at = timezone.utcnow() + self.start_date = None + self.end_date = None + if state == DagRunState.RUNNING: + if self._state in State.finished_dr_states: + self.start_date = timezone.utcnow() + else: + self.start_date = self.start_date or timezone.utcnow() + self.end_date = None + if self._state in State.unfinished_dr_states or self._state is None: + if state in State.finished_dr_states: + self.end_date = timezone.utcnow() self._state = state - self.end_date = timezone.utcnow() if self._state in State.finished_dr_states else None + else: if state == DagRunState.QUEUED: self.queued_at = timezone.utcnow() @@ -504,7 +568,7 @@ def get_task_instances( session: Session = NEW_SESSION, ) -> list[TI]: """ - Returns the task instances for this dag run. + Return the task instances for this dag run. Redirect to DagRun.fetch_task_instances method. Keep this method because it is widely used across the code. @@ -547,7 +611,7 @@ def fetch_task_instance( map_index: int = -1, ) -> TI | TaskInstancePydantic | None: """ - Returns the task instance specified by task_id for this dag run. + Return the task instance specified by task_id for this dag run. :param dag_id: the DAG id :param dag_run_id: the DAG run id diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 38b787cbe8fadb..01a84fc8834d94 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -461,7 +461,7 @@ def _refresh_from_db( *, task_instance: TaskInstance | TaskInstancePydantic, session: Session, lock_for_update: bool = False ) -> None: """ - Refreshes the task instance from the database based on the primary key. + Refresh the task instance from the database based on the primary key. :param task_instance: the task instance :param session: SQLAlchemy ORM Session @@ -531,7 +531,7 @@ def _set_duration(*, task_instance: TaskInstance | TaskInstancePydantic) -> None def _stats_tags(*, task_instance: TaskInstance | TaskInstancePydantic) -> dict[str, str]: """ - Returns task instance tags. + Return task instance tags. :param task_instance: the task instance @@ -943,7 +943,7 @@ def _get_previous_dagrun( session: Session | None = None, ) -> DagRun | None: """ - The DagRun that ran before this task instance's DagRun. + Return the DagRun that ran prior to this task instance's DagRun. :param task_instance: the task instance :param state: If passed, it only take into account instances of a specific state. @@ -983,7 +983,7 @@ def _get_previous_execution_date( session: Session, ) -> pendulum.DateTime | None: """ - The execution date from property previous_ti_success. + Get execution date from property previous_ti_success. :param task_instance: the task instance :param session: SQLAlchemy ORM Session @@ -1178,7 +1178,7 @@ def _get_previous_ti( state: DagRunState | None = None, ) -> TaskInstance | TaskInstancePydantic | None: """ - The task instance for the task that ran before this task instance. + Get task instance for the task that ran before this task instance. :param task_instance: the task instance :param state: If passed, it only take into account instances of a specific state. @@ -1436,7 +1436,7 @@ def try_number(self): @try_number.expression def try_number(cls): """ - This is what will be used by SQLAlchemy when filtering on try_number. + Return the expression to be used by SQLAlchemy when filtering on try_number. This is required because the override in the get_try_number function causes try_number values to be off by one when listing tasks in the UI. diff --git a/airflow/operators/python.py b/airflow/operators/python.py index 1b1453cc5ed50f..0f005f43b266aa 100644 --- a/airflow/operators/python.py +++ b/airflow/operators/python.py @@ -640,7 +640,7 @@ def _prepare_venv(self, venv_path: Path) -> None: ) def _calculate_cache_hash(self) -> tuple[str, str]: - """Helper to generate the hash of the cache folder to use. + """Generate the hash of the cache folder to use. The following factors are used as input for the hash: - (sorted) list of requirements @@ -666,7 +666,7 @@ def _calculate_cache_hash(self) -> tuple[str, str]: return requirements_hash[:8], hash_text def _ensure_venv_cache_exists(self, venv_cache_path: Path) -> Path: - """Helper to ensure a valid virtual environment is set up and will create inplace.""" + """Ensure a valid virtual environment is set up and will create inplace.""" cache_hash, hash_data = self._calculate_cache_hash() venv_path = venv_cache_path / f"venv-{cache_hash}" self.log.info("Python virtual environment will be cached in %s", venv_path) diff --git a/airflow/plugins_manager.py b/airflow/plugins_manager.py index 143e3af5707bc5..6514409ef493de 100644 --- a/airflow/plugins_manager.py +++ b/airflow/plugins_manager.py @@ -171,14 +171,14 @@ class AirflowPlugin: @classmethod def validate(cls): - """Validates that plugin has a name.""" + """Validate if plugin has a name.""" if not cls.name: raise AirflowPluginException("Your plugin needs a name.") @classmethod def on_load(cls, *args, **kwargs): """ - Executed when the plugin is loaded; This method is only called once during runtime. + Execute when the plugin is loaded; This method is only called once during runtime. :param args: If future arguments are passed in on call. :param kwargs: If future arguments are passed in on call. @@ -296,7 +296,7 @@ def load_providers_plugins(): def make_module(name: str, objects: list[Any]): - """Creates new module.""" + """Create new module.""" if not objects: return None log.debug("Creating module %s", name) diff --git a/airflow/providers/amazon/aws/executors/ecs/ecs_executor.py b/airflow/providers/amazon/aws/executors/ecs/ecs_executor.py index 2f0564ed9a3400..e6594e270f8be5 100644 --- a/airflow/providers/amazon/aws/executors/ecs/ecs_executor.py +++ b/airflow/providers/amazon/aws/executors/ecs/ecs_executor.py @@ -48,6 +48,7 @@ ) from airflow.providers.amazon.aws.hooks.ecs import EcsHook from airflow.utils import timezone +from airflow.utils.helpers import merge_dicts from airflow.utils.state import State if TYPE_CHECKING: @@ -408,8 +409,8 @@ def _run_task( The command and executor config will be placed in the container-override section of the JSON request before calling Boto3's "run_task" function. """ - run_task_api = self._run_task_kwargs(task_id, cmd, queue, exec_config) - boto_run_task = self.ecs.run_task(**run_task_api) + run_task_kwargs = self._run_task_kwargs(task_id, cmd, queue, exec_config) + boto_run_task = self.ecs.run_task(**run_task_kwargs) run_task_response = BotoRunTaskSchema().load(boto_run_task) return run_task_response @@ -421,17 +422,17 @@ def _run_task_kwargs( One last chance to modify Boto3's "run_task" kwarg params before it gets passed into the Boto3 client. """ - run_task_api = deepcopy(self.run_task_kwargs) - container_override = self.get_container(run_task_api["overrides"]["containerOverrides"]) + run_task_kwargs = deepcopy(self.run_task_kwargs) + run_task_kwargs = merge_dicts(run_task_kwargs, exec_config) + container_override = self.get_container(run_task_kwargs["overrides"]["containerOverrides"]) container_override["command"] = cmd - container_override.update(exec_config) # Inject the env variable to configure logging for containerized execution environment if "environment" not in container_override: container_override["environment"] = [] container_override["environment"].append({"name": "AIRFLOW_IS_EXECUTOR_CONTAINER", "value": "true"}) - return run_task_api + return run_task_kwargs def execute_async(self, key: TaskInstanceKey, command: CommandType, queue=None, executor_config=None): """Save the task to be executed in the next sync by inserting the commands into a queue.""" @@ -484,6 +485,11 @@ def _load_run_kwargs(self) -> dict: def get_container(self, container_list): """Searches task list for core Airflow container.""" for container in container_list: - if container["name"] == self.container_name: - return container + try: + if container["name"] == self.container_name: + return container + except KeyError: + raise EcsExecutorException( + 'container "name" must be provided in "containerOverrides" configuration' + ) raise KeyError(f"No such container found by container name: {self.container_name}") diff --git a/airflow/providers/fab/auth_manager/decorators/auth.py b/airflow/providers/fab/auth_manager/decorators/auth.py index 95f97c8e795904..7089be08fc56df 100644 --- a/airflow/providers/fab/auth_manager/decorators/auth.py +++ b/airflow/providers/fab/auth_manager/decorators/auth.py @@ -67,7 +67,7 @@ def decorated(*args, **kwargs): def _has_access_fab(permissions: Sequence[tuple[str, str]] | None = None) -> Callable[[T], T]: """ - Factory for decorator that checks current user's permissions against required permissions. + Check current user's permissions against required permissions. This decorator is only kept for backward compatible reasons. The decorator ``airflow.www.auth.has_access``, which redirects to this decorator, is widely used in user plugins. diff --git a/airflow/providers/fab/auth_manager/fab_auth_manager.py b/airflow/providers/fab/auth_manager/fab_auth_manager.py index dfa53ef78b5b08..696709ae6cad8e 100644 --- a/airflow/providers/fab/auth_manager/fab_auth_manager.py +++ b/airflow/providers/fab/auth_manager/fab_auth_manager.py @@ -446,7 +446,7 @@ def _get_fab_resource_types(dag_access_entity: DagAccessEntity) -> tuple[str, .. def _resource_name_for_dag(self, dag_id: str) -> str: """ - Returns the FAB resource name for a DAG id. + Return the FAB resource name for a DAG id. :param dag_id: the DAG id diff --git a/airflow/providers/fab/auth_manager/security_manager/override.py b/airflow/providers/fab/auth_manager/security_manager/override.py index 9fe89f8a69edbf..6f5c0f72c66985 100644 --- a/airflow/providers/fab/auth_manager/security_manager/override.py +++ b/airflow/providers/fab/auth_manager/security_manager/override.py @@ -531,7 +531,7 @@ def auth_rate_limit(self) -> str: @property def auth_role_public(self): - """Gets the public role.""" + """Get the public role.""" return self.appbuilder.app.config["AUTH_ROLE_PUBLIC"] @property @@ -571,7 +571,7 @@ def auth_ldap_tls_demand(self): @property def auth_ldap_server(self): - """Gets the LDAP server object.""" + """Get the LDAP server object.""" return self.appbuilder.get_app.config["AUTH_LDAP_SERVER"] @property @@ -650,7 +650,7 @@ def api_login_allow_multiple_providers(self): @property def auth_username_ci(self): - """Gets the auth username for CI.""" + """Get the auth username for CI.""" return self.appbuilder.get_app.config.get("AUTH_USERNAME_CI", True) @property @@ -685,7 +685,7 @@ def auth_roles_sync_at_login(self) -> bool: @property def auth_role_admin(self): - """Gets the admin role.""" + """Get the admin role.""" return self.appbuilder.get_app.config["AUTH_ROLE_ADMIN"] @property @@ -697,7 +697,7 @@ def oauth_whitelists(self): return self.oauth_allow_list def create_builtin_roles(self): - """Returns FAB builtin roles.""" + """Return FAB builtin roles.""" return self.appbuilder.app.config.get("FAB_ROLES", {}) @property @@ -1445,7 +1445,7 @@ def add_user( password="", hashed_password="", ): - """Generic function to create user.""" + """Create a user.""" try: user = self.user_model() user.first_name = first_name @@ -1504,7 +1504,7 @@ def add_register_user(self, username, first_name, last_name, email, password="", return None def find_user(self, username=None, email=None): - """Finds user by username or email.""" + """Find user by username or email.""" if username: try: if self.auth_username_ci: @@ -1549,7 +1549,7 @@ def update_user(self, user: User) -> bool: def del_register_user(self, register_user): """ - Deletes registration object from database. + Delete registration object from database. :param register_user: RegisterUser object to delete """ @@ -1598,7 +1598,7 @@ def update_user_auth_stat(self, user, success=True): def get_action(self, name: str) -> Action: """ - Gets an existing action record. + Get an existing action record. :param name: name """ @@ -1606,7 +1606,7 @@ def get_action(self, name: str) -> Action: def create_action(self, name): """ - Adds an action to the backend, model action. + Add an action to the backend, model action. :param name: name of the action: 'can_add','can_edit' etc... @@ -1626,7 +1626,7 @@ def create_action(self, name): def delete_action(self, name: str) -> bool: """ - Deletes a permission action. + Delete a permission action. :param name: Name of action to delete (e.g. can_read). """ @@ -1659,7 +1659,7 @@ def delete_action(self, name: str) -> bool: def get_resource(self, name: str) -> Resource: """ - Returns a resource record by name, if it exists. + Return a resource record by name, if it exists. :param name: Name of resource """ @@ -1685,12 +1685,12 @@ def create_resource(self, name) -> Resource: return resource def get_all_resources(self) -> list[Resource]: - """Gets all existing resource records.""" + """Get all existing resource records.""" return self.get_session.query(self.resource_model).all() def delete_resource(self, name: str) -> bool: """ - Deletes a Resource from the backend. + Delete a Resource from the backend. :param name: name of the resource @@ -1728,7 +1728,7 @@ def get_permission( resource_name: str, ) -> Permission | None: """ - Gets a permission made with the given action->resource pair, if the permission already exists. + Get a permission made with the given action->resource pair, if the permission already exists. :param action_name: Name of action :param resource_name: Name of resource @@ -1753,7 +1753,7 @@ def get_resource_permissions(self, resource: Resource) -> Permission: def create_permission(self, action_name, resource_name) -> Permission | None: """ - Adds a permission on a resource to the backend. + Add a permission on a resource to the backend. :param action_name: name of the action to add: 'can_add','can_edit' etc... @@ -1781,7 +1781,7 @@ def create_permission(self, action_name, resource_name) -> Permission | None: def delete_permission(self, action_name: str, resource_name: str) -> None: """ - Deletes the permission linking an action->resource pair. + Delete the permission linking an action->resource pair. Doesn't delete the underlying action or resource. @@ -1846,7 +1846,7 @@ def remove_permission_from_role(self, role: Role, permission: Permission) -> Non self.get_session.rollback() def get_oid_identity_url(self, provider_name: str) -> str | None: - """Returns the OIDC identity provider URL.""" + """Return the OIDC identity provider URL.""" for provider in self.openid_providers: if provider.get("name") == provider_name: return provider.get("url") @@ -2091,7 +2091,7 @@ def oauth_user_info_getter( func: Callable[[AirflowSecurityManagerV2, str, dict[str, Any] | None], dict[str, Any]], ): """ - Decorator function to be the OAuth user info getter for all the providers. + Get OAuth user info for all the providers. Receives provider and response return a dict with the information returned from the provider. The returned user info dict should have its keys with the same name as the User Model. @@ -2210,7 +2210,7 @@ def get_oauth_user_info(self, provider: str, resp: dict[str, Any]) -> dict[str, @staticmethod def oauth_token_getter(): - """Authentication (OAuth) token getter function.""" + """Get authentication (OAuth) token.""" token = session.get("oauth") log.debug("Token Get: %s", token) return token @@ -2220,7 +2220,7 @@ def check_authorization( perms: Sequence[tuple[str, str]] | None = None, dag_id: str | None = None, ) -> bool: - """Checks that the logged in user has the specified permissions.""" + """Check the logged-in user has the specified permissions.""" if not perms: return True @@ -2254,7 +2254,7 @@ def set_oauth_session(self, provider, oauth_response): def get_oauth_token_key_name(self, provider): """ - Returns the token_key name for the oauth provider. + Return the token_key name for the oauth provider. If none is configured defaults to oauth_token this is configured using OAUTH_PROVIDERS and token_key key. @@ -2275,7 +2275,7 @@ def get_oauth_token_secret_name(self, provider): def auth_user_oauth(self, userinfo): """ - Method for authenticating user with OAuth. + Authenticate user with OAuth. :userinfo: dict with user information (keys are the same as User model columns) @@ -2608,7 +2608,7 @@ def _get_user_permission_resources( return result def _has_access_builtin_roles(self, role, action_name: str, resource_name: str) -> bool: - """Checks permission on builtin role.""" + """Check permission on builtin role.""" perms = self.builtin_roles.get(role.name, []) for _resource_name, _action_name in perms: if re2.match(_resource_name, resource_name) and re2.match(_action_name, action_name): @@ -2647,7 +2647,7 @@ def _get_all_non_dag_permissions(self) -> dict[tuple[str, str], Permission]: """ Get permissions except those that are for specific DAGs. - Returns a dict with a key of (action_name, resource_name) and value of permission + Return a dict with a key of (action_name, resource_name) and value of permission with all permissions except those that are for specific DAGs. """ return { @@ -2689,7 +2689,7 @@ def _get_root_dag_id(self, dag_id: str) -> str: @staticmethod def _cli_safe_flash(text: str, level: str) -> None: - """Shows a flash in a web context or prints a message if not.""" + """Show a flash in a web context or prints a message if not.""" if has_request_context(): flash(Markup(text), level) else: diff --git a/airflow/providers/pinecone/hooks/pinecone.py b/airflow/providers/pinecone/hooks/pinecone.py index f15605556cf871..6a116250f49228 100644 --- a/airflow/providers/pinecone/hooks/pinecone.py +++ b/airflow/providers/pinecone/hooks/pinecone.py @@ -45,7 +45,7 @@ class PineconeHook(BaseHook): @classmethod def get_connection_form_widgets(cls) -> dict[str, Any]: - """Returns connection widgets to add to connection form.""" + """Return connection widgets to add to connection form.""" from flask_appbuilder.fieldwidgets import BS3TextFieldWidget from flask_babel import lazy_gettext from wtforms import StringField @@ -60,7 +60,7 @@ def get_connection_form_widgets(cls) -> dict[str, Any]: @classmethod def get_ui_field_behaviour(cls) -> dict[str, Any]: - """Returns custom field behaviour.""" + """Return custom field behaviour.""" return { "hidden_fields": ["port", "schema"], "relabeling": {"login": "Pinecone Environment", "password": "Pinecone API key"}, @@ -108,7 +108,7 @@ def upsert( **kwargs: Any, ) -> UpsertResponse: """ - The upsert operation writes vectors into a namespace. + Write vectors into a namespace. If a new value is upserted for an existing vector id, it will overwrite the previous value. @@ -204,7 +204,7 @@ def delete_index(index_name: str, timeout: int | None = None) -> None: @staticmethod def configure_index(index_name: str, replicas: int | None = None, pod_type: str | None = "") -> None: """ - Changes current configuration of the index. + Change the current configuration of the index. :param index_name: The name of the index to configure. :param replicas: The new number of replicas. @@ -258,7 +258,7 @@ def query_vector( sparse_vector: SparseValues | dict[str, list[float] | list[int]] | None = None, ) -> QueryResponse: """ - The Query operation searches a namespace, using a query vector. + Search a namespace using query vector. It retrieves the ids of the most similar items in a namespace, along with their similarity scores. API reference: https://docs.pinecone.io/reference/query @@ -288,7 +288,7 @@ def query_vector( @staticmethod def _chunks(iterable: list[Any], batch_size: int = 100) -> Any: - """Helper function to break an iterable into chunks of size batch_size.""" + """Break an iterable into chunks of size batch_size.""" it = iter(iterable) chunk = tuple(itertools.islice(it, batch_size)) while chunk: @@ -329,7 +329,7 @@ def describe_index_stats( **kwargs: Any, ) -> DescribeIndexStatsResponse: """ - Describes the index statistics. + Describe the index statistics. Returns statistics about the index's contents. For example: The vector count per namespace and the number of dimensions. diff --git a/airflow/providers/teradata/hooks/teradata.py b/airflow/providers/teradata/hooks/teradata.py index 5f5d1ed8fbd5c4..1ede21203d6f43 100644 --- a/airflow/providers/teradata/hooks/teradata.py +++ b/airflow/providers/teradata/hooks/teradata.py @@ -79,7 +79,7 @@ def __init__( super().__init__(*args, schema=database, **kwargs) def get_conn(self) -> TeradataConnection: - """Creates and returns a Teradata Connection object using teradatasql client. + """Create and return a Teradata Connection object using teradatasql client. Establishes connection to a Teradata SQL database using config corresponding to teradata_conn_id. @@ -96,7 +96,7 @@ def bulk_insert_rows( target_fields: list[str] | None = None, commit_every: int = 5000, ): - """A bulk insert of records for Teradata SQL Database. + """Insert bulk of records into Teradata SQL Database. This uses prepared statements via `executemany()`. For best performance, pass in `rows` as an iterator. @@ -142,7 +142,7 @@ def bulk_insert_rows( conn.close() # type: ignore[attr-defined] def _get_conn_config_teradatasql(self) -> dict[str, Any]: - """Returns set of config params required for connecting to Teradata DB using teradatasql client.""" + """Return set of config params required for connecting to Teradata DB using teradatasql client.""" conn: Connection = self.get_connection(getattr(self, self.conn_name_attr)) conn_config = { "host": conn.host or "localhost", @@ -174,7 +174,7 @@ def _get_conn_config_teradatasql(self) -> dict[str, Any]: return conn_config def get_sqlalchemy_engine(self, engine_kwargs=None): - """Returns a connection object using sqlalchemy.""" + """Return a connection object using sqlalchemy.""" conn: Connection = self.get_connection(getattr(self, self.conn_name_attr)) link = f"teradatasql://{conn.login}:{conn.password}@{conn.host}" connection = sqlalchemy.create_engine(link) @@ -182,7 +182,7 @@ def get_sqlalchemy_engine(self, engine_kwargs=None): @staticmethod def get_ui_field_behaviour() -> dict: - """Returns custom field behaviour.""" + """Return custom field behaviour.""" import json return { diff --git a/airflow/providers_manager.py b/airflow/providers_manager.py index 075473796bc8fa..1f1fe397b9785e 100644 --- a/airflow/providers_manager.py +++ b/airflow/providers_manager.py @@ -146,7 +146,7 @@ def _read_schema_from_resources_or_local_file(filename: str) -> dict: def _create_provider_info_schema_validator(): - """Creates JSON schema validator from the provider_info.schema.json.""" + """Create JSON schema validator from the provider_info.schema.json.""" import jsonschema schema = _read_schema_from_resources_or_local_file("provider_info.schema.json") @@ -156,7 +156,7 @@ def _create_provider_info_schema_validator(): def _create_customized_form_field_behaviours_schema_validator(): - """Creates JSON schema validator from the customized_form_field_behaviours.schema.json.""" + """Create JSON schema validator from the customized_form_field_behaviours.schema.json.""" import jsonschema schema = _read_schema_from_resources_or_local_file("customized_form_field_behaviours.schema.json") @@ -305,7 +305,7 @@ def _correctness_check( provider_package: str, class_name: str, provider_info: ProviderInfo ) -> type[BaseHook] | None: """ - Performs coherence check on provider classes. + Perform coherence check on provider classes. For apache-airflow providers - it checks if it starts with appropriate package. For all providers it tries to import the provider - checking that there are no exceptions during importing. @@ -408,7 +408,7 @@ def initialization_stack_trace() -> str | None: return ProvidersManager._initialization_stack_trace def __init__(self): - """Initializes the manager.""" + """Initialize the manager.""" super().__init__() ProvidersManager._initialized = True ProvidersManager._initialization_stack_trace = "".join(traceback.format_stack(inspect.currentframe())) @@ -445,7 +445,7 @@ def __init__(self): self._init_airflow_core_hooks() def _init_airflow_core_hooks(self): - """Initializes the hooks dict with default hooks from Airflow core.""" + """Initialize the hooks dict with default hooks from Airflow core.""" core_dummy_hooks = { "generic": "Generic", "email": "Email", @@ -563,7 +563,7 @@ def initialize_providers_configuration(self): def _initialize_providers_configuration(self): """ - Internal method to initialize providers configuration information. + Initialize providers configuration information. Should be used if we do not want to trigger caching for ``initialize_providers_configuration`` method. In some cases we might want to make sure that the configuration is initialized, but we do not want @@ -626,7 +626,7 @@ def _discover_all_providers_from_packages(self) -> None: def _discover_all_airflow_builtin_providers_from_local_sources(self) -> None: """ - Finds all built-in airflow providers if airflow is run from the local sources. + Find all built-in airflow providers if airflow is run from the local sources. It finds `provider.yaml` files for all such providers and registers the providers using those. @@ -654,7 +654,7 @@ def _discover_all_airflow_builtin_providers_from_local_sources(self) -> None: def _add_provider_info_from_local_source_files_on_path(self, path) -> None: """ - Finds all the provider.yaml files in the directory specified. + Find all the provider.yaml files in the directory specified. :param path: path where to look for provider.yaml files """ @@ -672,7 +672,7 @@ def _add_provider_info_from_local_source_files_on_path(self, path) -> None: def _add_provider_info_from_local_source_file(self, path, package_name) -> None: """ - Parses found provider.yaml file and adds found provider to the dictionary. + Parse found provider.yaml file and adds found provider to the dictionary. :param path: full file path of the provider.yaml file :param package_name: name of the package @@ -1069,7 +1069,7 @@ def _add_customized_fields(self, package_name: str, hook_class: type, customized ) def _discover_auth_managers(self) -> None: - """Retrieves all auth managers defined in the providers.""" + """Retrieve all auth managers defined in the providers.""" for provider_package, provider in self._provider_dict.items(): if provider.data.get("auth-managers"): for auth_manager_class_name in provider.data["auth-managers"]: @@ -1077,7 +1077,7 @@ def _discover_auth_managers(self) -> None: self._auth_manager_class_name_set.add(auth_manager_class_name) def _discover_notifications(self) -> None: - """Retrieves all notifications defined in the providers.""" + """Retrieve all notifications defined in the providers.""" for provider_package, provider in self._provider_dict.items(): if provider.data.get("notifications"): for notification_class_name in provider.data["notifications"]: @@ -1085,7 +1085,7 @@ def _discover_notifications(self) -> None: self._notification_info_set.add(notification_class_name) def _discover_extra_links(self) -> None: - """Retrieves all extra links defined in the providers.""" + """Retrieve all extra links defined in the providers.""" for provider_package, provider in self._provider_dict.items(): if provider.data.get("extra-links"): for extra_link_class_name in provider.data["extra-links"]: @@ -1149,7 +1149,7 @@ def _discover_plugins(self) -> None: @provider_info_cache("triggers") def initialize_providers_triggers(self): - """Initialization of providers triggers.""" + """Initialize providers triggers.""" self.initialize_providers_list() for provider_package, provider in self._provider_dict.items(): for trigger in provider.data.get("triggers", []): diff --git a/airflow/serialization/serde.py b/airflow/serialization/serde.py index a214acc9a6677d..fd7eb33af72846 100644 --- a/airflow/serialization/serde.py +++ b/airflow/serialization/serde.py @@ -288,20 +288,20 @@ def _convert(old: dict) -> dict: def _match(classname: str) -> bool: - """Checks if the given classname matches a path pattern either using glob format or regexp format.""" + """Check if the given classname matches a path pattern either using glob format or regexp format.""" return _match_glob(classname) or _match_regexp(classname) @functools.lru_cache(maxsize=None) def _match_glob(classname: str): - """Checks if the given classname matches a pattern from allowed_deserialization_classes using glob syntax.""" + """Check if the given classname matches a pattern from allowed_deserialization_classes using glob syntax.""" patterns = _get_patterns() return any(fnmatch(classname, p.pattern) for p in patterns) @functools.lru_cache(maxsize=None) def _match_regexp(classname: str): - """Checks if the given classname matches a pattern from allowed_deserialization_classes_regexp using regexp.""" + """Check if the given classname matches a pattern from allowed_deserialization_classes_regexp using regexp.""" patterns = _get_regexp_patterns() return any(p.match(classname) is not None for p in patterns) diff --git a/airflow/utils/file.py b/airflow/utils/file.py index 7e15eeb2f8d72c..c66358a10aebf8 100644 --- a/airflow/utils/file.py +++ b/airflow/utils/file.py @@ -385,7 +385,7 @@ def iter_airflow_imports(file_path: str) -> Generator[str, None, None]: def get_unique_dag_module_name(file_path: str) -> str: - """Returns a unique module name in the format unusual_prefix_{sha1 of module's file path}_{original module name}.""" + """Return a unique module name in the format unusual_prefix_{sha1 of module's file path}_{original module name}.""" if isinstance(file_path, str): path_hash = hashlib.sha1(file_path.encode("utf-8")).hexdigest() org_mod_name = Path(file_path).stem diff --git a/airflow/utils/log/task_context_logger.py b/airflow/utils/log/task_context_logger.py index 84ed207e3ae9cf..46e8cf8cee3823 100644 --- a/airflow/utils/log/task_context_logger.py +++ b/airflow/utils/log/task_context_logger.py @@ -65,7 +65,7 @@ def _should_enable(self) -> bool: @staticmethod def _get_task_handler() -> FileTaskHandler | None: - """Returns the task handler that supports task context logging.""" + """Return the task handler that supports task context logging.""" handlers = [ handler for handler in logging.getLogger("airflow.task").handlers diff --git a/airflow/utils/sqlalchemy.py b/airflow/utils/sqlalchemy.py index 2dc495811ae7b2..6ce0d00207ae00 100644 --- a/airflow/utils/sqlalchemy.py +++ b/airflow/utils/sqlalchemy.py @@ -271,6 +271,8 @@ def process(value): def compare_values(self, x, y): """ + Compare x and y using self.comparator if available. Else, use __eq__. + The TaskInstance.executor_config attribute is a pickled object that may contain kubernetes objects. If the installed library version has changed since the object was originally pickled, diff --git a/airflow/www/auth.py b/airflow/www/auth.py index a34621d56c416b..39c8444f993c2c 100644 --- a/airflow/www/auth.py +++ b/airflow/www/auth.py @@ -88,7 +88,7 @@ def has_access(permissions: Sequence[tuple[str, str]] | None = None) -> Callable def has_access_with_pk(f): """ - This decorator is used to check permissions on views. + Check permissions on views. The implementation is very similar from https://github.com/dpgaspar/Flask-AppBuilder/blob/c6fecdc551629e15467fde5d06b4437379d90592/flask_appbuilder/security/decorators.py#L134 @@ -345,5 +345,5 @@ def decorated(*args, **kwargs): def has_access_view(access_view: AccessView = AccessView.WEBSITE) -> Callable[[T], T]: - """Decorator that checks current user's permissions to access the website.""" + """Check current user's permissions to access the website.""" return _has_access_no_details(lambda: get_auth_manager().is_authorized_view(access_view=access_view)) diff --git a/airflow/www/blueprints.py b/airflow/www/blueprints.py index 0312a9ffa71639..fda6b65397b66b 100644 --- a/airflow/www/blueprints.py +++ b/airflow/www/blueprints.py @@ -24,5 +24,5 @@ @routes.route("/") def index(): - """Main Airflow page.""" + """Return main Airflow page.""" return redirect(url_for("Airflow.index")) diff --git a/airflow/www/views.py b/airflow/www/views.py index 6e6caeba3c4072..16253078905596 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -3994,7 +3994,7 @@ def delete(self, pk): @expose("/action_post", methods=["POST"]) def action_post(self): """ - Action method to handle multiple records selected from a list view. + Handle multiple records selected from a list view. Same implementation as https://github.com/dpgaspar/Flask-AppBuilder/blob/2c5763371b81cd679d88b9971ba5d1fc4d71d54b/flask_appbuilder/views.py#L677 diff --git a/contributing-docs/06_development_environments.rst b/contributing-docs/06_development_environments.rst index e442ed735a1f12..c467d6932855c1 100644 --- a/contributing-docs/06_development_environments.rst +++ b/contributing-docs/06_development_environments.rst @@ -32,7 +32,7 @@ in `07_local_virtualenv.rst <07_local_virtualenv.rst>`__. Benefits: - Packages are installed locally. No container environment is required. -- You can benefit from local debugging within your IDE. You can follow the `Contributors quick start `__ +- You can benefit from local debugging within your IDE. You can follow the `Local and remote debugging in IDE <07_local_virtualenv.rst#local-and-remote-debugging-in-ide>`__ to set up your local virtualenv and connect your IDE with the environment. - With the virtualenv in your IDE, you can benefit from auto completion and running tests directly from the IDE. diff --git a/dev/breeze/src/airflow_breeze/commands/developer_commands.py b/dev/breeze/src/airflow_breeze/commands/developer_commands.py index 92882a781a367d..e9eb6290c56d9b 100644 --- a/dev/breeze/src/airflow_breeze/commands/developer_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/developer_commands.py @@ -656,7 +656,11 @@ def build_docs( fix_ownership_using_docker() if result.returncode == 0: get_console().print( - "[info]Start the webserver in breeze and view the built docs at http://localhost:28080/docs/[/]" + "[info]To view the built documentation, you have two options:\n\n" + "1. Start the webserver in breeze and access the built docs at " + "http://localhost:28080/docs/\n" + "2. Alternatively, you can run ./docs/start_docs_server.sh for a lighter resource option and view" + "the built docs at http://localhost:8000" ) sys.exit(result.returncode) diff --git a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py index 5a4e569e42a7bb..35d8e56d809ad4 100644 --- a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py +++ b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py @@ -567,6 +567,7 @@ def fix_ownership_using_docker(quiet: bool = False): f"DOCKER_IS_ROOTLESS={is_docker_rootless()}", "-e", f"VERBOSE_COMMANDS={str(not quiet).lower()}", + "--rm", "-t", OWNERSHIP_CLEANUP_DOCKER_TAG, "/opt/airflow/scripts/in_container/run_fix_ownership.py", diff --git a/dev/breeze/tests/test_selective_checks.py b/dev/breeze/tests/test_selective_checks.py index 89a7374f924329..77f2fb17f23a0c 100644 --- a/dev/breeze/tests/test_selective_checks.py +++ b/dev/breeze/tests/test_selective_checks.py @@ -1225,7 +1225,7 @@ def test_upgrade_to_newer_dependencies( "docs-list-as-string": "apache-airflow amazon apache.drill apache.druid apache.hive " "apache.impala apache.pinot common.sql databricks elasticsearch " "exasol google jdbc microsoft.mssql mysql odbc openlineage " - "oracle pgvector postgres presto slack snowflake sqlite trino vertica", + "oracle pgvector postgres presto slack snowflake sqlite teradata trino vertica", }, id="Common SQL provider package python files changed", ), diff --git a/docs/README.rst b/docs/README.rst index e16f22499e8945..d5cddec4864f54 100644 --- a/docs/README.rst +++ b/docs/README.rst @@ -162,19 +162,18 @@ Running the Docs Locally ------------------------ After you build the documentation, you can check the formatting, style, and documentation build at ``http://localhost:28080/docs/`` -by starting a Breeze environment or by running the following command from the root directory. - -You need to have Python installed to run the command: +by starting a Breeze environment. Alternatively, you can run the following command from the root directory: .. code-block:: bash docs/start_doc_server.sh +This command requires Python to be installed. This method is lighter on the system resources as you do not need to +launch the webserver just to view docs. -Then, view your docs at ``localhost:8000``. If you use a virtual machine, like WSL2, -you need to find the WSL2 machine IP address and replace "0.0.0.0" in your browser with it. The address looks like -``http://n.n.n.n:8000``, where n.n.n.n is the IP of the WSL2. - +Once the server is running, you can view your documentation at http://localhost:8000. If you're using a virtual machine +like WSL2, you'll need to find the IP address of the WSL2 machine and replace "0.0.0.0" in your browser with it. +The address will look like http://n.n.n.n:8000, where n.n.n.n is the IP of the WSL2 machine. Cross-referencing syntax ======================== diff --git a/docs/apache-airflow-providers-amazon/executors/ecs-executor.rst b/docs/apache-airflow-providers-amazon/executors/ecs-executor.rst index a6062a630437bd..d8d3764f5e0429 100644 --- a/docs/apache-airflow-providers-amazon/executors/ecs-executor.rst +++ b/docs/apache-airflow-providers-amazon/executors/ecs-executor.rst @@ -73,6 +73,9 @@ In the case of conflicts, the order of precedence from lowest to highest is: 3. Load any values provided in the RUN_TASK_KWARGS option if one is provided. +.. note:: + ``exec_config`` is an optional parameter that can be provided to operators. It is a dictionary type and in the context of the ECS Executor it represents a ``run_task_kwargs`` configuration which is then updated over-top of the ``run_task_kwargs`` specified in Airflow config above (if present). It is a recursive update which essentially applies Python update to each nested dictionary in the configuration. Loosely approximated as: ``run_task_kwargs.update(exec_config)`` + Required config options: ~~~~~~~~~~~~~~~~~~~~~~~~ @@ -88,7 +91,7 @@ Optional config options: - ASSIGN_PUBLIC_IP - Whether to assign a public IP address to the containers launched by the ECS executor. Defaults to "False". -- CONN_ID - The Airflow connection (i.e. credentials) used by the ECS +- AWS_CONN_ID - The Airflow connection (i.e. credentials) used by the ECS executor to make API calls to AWS ECS. Defaults to "aws_default". - LAUNCH_TYPE - Launch type can either be 'FARGATE' OR 'EC2'. Defaults to "FARGATE". @@ -113,6 +116,9 @@ For a more detailed description of available options, including type hints and examples, see the ``config_templates`` folder in the Amazon provider package. +.. note:: + ``exec_config`` is an optional parameter that can be provided to operators. It is a dictionary type and in the context of the ECS Executor it represents a ``run_task_kwargs`` configuration which is then updated over-top of the ``run_task_kwargs`` specified in Airflow config above (if present). It is a recursive update which essentially applies Python update to each nested dictionary in the configuration. Loosely approximated as: ``run_task_kwargs.update(exec_config)`` + .. _dockerfile_for_ecs_executor: Dockerfile for ECS Executor diff --git a/pyproject.toml b/pyproject.toml index ec41e835f5cd72..aa2acfc625004a 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -1357,22 +1357,6 @@ combine-as-imports = true "tests/providers/elasticsearch/log/elasticmock/utilities/__init__.py" = ["E402"] # All the modules which do not follow D401 yet, please remove as soon as it becomes compatible -"airflow/auth/managers/utils/fab.py" = ["D401"] -"airflow/decorators/bash.py" = ["D401"] -"airflow/executors/debug_executor.py" = ["D401"] -"airflow/models/baseoperator.py" = ["D401"] -"airflow/models/dagrun.py" = ["D401"] -"airflow/models/taskinstance.py" = ["D401"] -"airflow/operators/python.py" = ["D401"] -"airflow/plugins_manager.py" = ["D401"] -"airflow/providers_manager.py" = ["D401"] -"airflow/serialization/serde.py" = ["D401"] -"airflow/utils/log/task_context_logger.py" = ["D401"] -"airflow/utils/sqlalchemy.py" = ["D401"] -"airflow/www/auth.py" = ["D401"] -"airflow/www/blueprints.py" = ["D401"] -"airflow/www/views.py" = ["D401"] -"airflow/utils/file.py" = ["D401"] "airflow/providers/airbyte/hooks/airbyte.py" = ["D401"] "airflow/providers/airbyte/operators/airbyte.py" = ["D401"] "airflow/providers/airbyte/sensors/airbyte.py" = ["D401"] @@ -1396,9 +1380,6 @@ combine-as-imports = true "airflow/providers/common/io/xcom/backend.py" = ["D401"] "airflow/providers/databricks/hooks/databricks.py" = ["D401"] "airflow/providers/databricks/operators/databricks.py" = ["D401"] -"airflow/providers/fab/auth_manager/decorators/auth.py" = ["D401"] -"airflow/providers/fab/auth_manager/fab_auth_manager.py" = ["D401"] -"airflow/providers/fab/auth_manager/security_manager/override.py" = ["D401"] "airflow/providers/google/cloud/hooks/automl.py" = ["D401"] "airflow/providers/google/cloud/hooks/bigquery.py" = ["D401"] "airflow/providers/google/cloud/hooks/bigquery_dts.py" = ["D401"] @@ -1569,7 +1550,6 @@ combine-as-imports = true "airflow/providers/pagerduty/hooks/pagerduty.py" = ["D401"] "airflow/providers/pagerduty/hooks/pagerduty_events.py" = ["D401"] "airflow/providers/papermill/hooks/kernel.py" = ["D401"] -"airflow/providers/pinecone/hooks/pinecone.py" = ["D401"] "airflow/providers/postgres/hooks/postgres.py" = ["D401"] "airflow/providers/presto/hooks/presto.py" = ["D401"] "airflow/providers/qdrant/hooks/qdrant.py" = ["D401"] diff --git a/tests/api_connexion/endpoints/test_dag_run_endpoint.py b/tests/api_connexion/endpoints/test_dag_run_endpoint.py index 045b5392f5f0ba..9f3d0666bd0626 100644 --- a/tests/api_connexion/endpoints/test_dag_run_endpoint.py +++ b/tests/api_connexion/endpoints/test_dag_run_endpoint.py @@ -1600,11 +1600,11 @@ def test_should_respond_200(self, state, run_type, dag_maker, session): "conf": {}, "dag_id": dag_id, "dag_run_id": dag_run_id, - "end_date": dr.end_date.isoformat(), + "end_date": dr.end_date.isoformat() if state != State.QUEUED else None, "execution_date": dr.execution_date.isoformat(), "external_trigger": False, "logical_date": dr.execution_date.isoformat(), - "start_date": dr.start_date.isoformat(), + "start_date": dr.start_date.isoformat() if state != State.QUEUED else None, "state": state, "data_interval_start": dr.data_interval_start.isoformat(), "data_interval_end": dr.data_interval_end.isoformat(), diff --git a/tests/api_experimental/client/test_local_client.py b/tests/api_experimental/client/test_local_client.py index b02a5a5c422972..91a81a0cafaee3 100644 --- a/tests/api_experimental/client/test_local_client.py +++ b/tests/api_experimental/client/test_local_client.py @@ -135,13 +135,11 @@ def test_trigger_dag(self, mock): # test output queued_at = pendulum.now() - started_at = pendulum.now() mock.return_value = DagRun( dag_id=test_dag_id, run_id=run_id, queued_at=queued_at, execution_date=EXECDATE, - start_date=started_at, external_trigger=True, state=DagRunState.QUEUED, conf={}, @@ -159,7 +157,7 @@ def test_trigger_dag(self, mock): "last_scheduling_decision": None, "logical_date": EXECDATE, "run_type": DagRunType.MANUAL, - "start_date": started_at, + "start_date": None, "state": DagRunState.QUEUED, } dag_run = self.client.trigger_dag(dag_id=test_dag_id) diff --git a/tests/api_experimental/common/test_mark_tasks.py b/tests/api_experimental/common/test_mark_tasks.py index 47c10fa1853956..9b28136bba2797 100644 --- a/tests/api_experimental/common/test_mark_tasks.py +++ b/tests/api_experimental/common/test_mark_tasks.py @@ -555,20 +555,28 @@ def _verify_dag_run_state(self, dag, date, state): assert dr.get_state() == state @provide_session - def _verify_dag_run_dates(self, dag, date, state, middle_time, session=None): + def _verify_dag_run_dates(self, dag, date, state, middle_time=None, old_end_date=None, session=None): # When target state is RUNNING, we should set start_date, # otherwise we should set end_date. DR = DagRun dr = session.query(DR).filter(DR.dag_id == dag.dag_id, DR.execution_date == date).one() if state == State.RUNNING: # Since the DAG is running, the start_date must be updated after creation - assert dr.start_date > middle_time + if middle_time: + assert dr.start_date > middle_time # If the dag is still running, we don't have an end date assert dr.end_date is None else: - # If the dag is not running, there must be an end time - assert dr.start_date < middle_time - assert dr.end_date > middle_time + # If the dag is not running, there must be an end time, + # and the end time must not be changed if it has already been set. + if dr.start_date and middle_time: + assert dr.start_date < middle_time + if dr.end_date: + if old_end_date: + assert dr.end_date == old_end_date + else: + if middle_time: + assert dr.end_date > middle_time def test_set_running_dag_run_to_success(self): date = self.execution_dates[0] @@ -599,30 +607,42 @@ def test_set_running_dag_run_to_failed(self): assert dr.get_task_instance("run_after_loop").state == State.FAILED self._verify_dag_run_dates(self.dag1, date, State.FAILED, middle_time) - @pytest.mark.parametrize( - "dag_run_alter_function, new_state", - [(set_dag_run_state_to_running, State.RUNNING), (set_dag_run_state_to_queued, State.QUEUED)], - ) - def test_set_running_dag_run_to_activate_state(self, dag_run_alter_function: Callable, new_state: State): + def test_set_running_dag_run_to_running_state(self): + date = self.execution_dates[0] # type: ignore + dr = self._create_test_dag_run(State.RUNNING, date) + self._set_default_task_instance_states(dr) + + altered = set_dag_run_state_to_running(dag=self.dag1, run_id=dr.run_id, commit=True) # type: ignore + + # None of the tasks should be altered, only the dag itself + assert len(altered) == 0 + new_state = State.RUNNING + self._verify_dag_run_state(self.dag1, date, new_state) # type: ignore + self._verify_task_instance_states_remain_default(dr) + self._verify_dag_run_dates(self.dag1, date, new_state) # type: ignore + + def test_set_running_dag_run_to_queued_state(self): date = self.execution_dates[0] # type: ignore dr = self._create_test_dag_run(State.RUNNING, date) middle_time = timezone.utcnow() self._set_default_task_instance_states(dr) - altered = dag_run_alter_function(dag=self.dag1, run_id=dr.run_id, commit=True) # type: ignore + altered = set_dag_run_state_to_queued(dag=self.dag1, run_id=dr.run_id, commit=True) # type: ignore # None of the tasks should be altered, only the dag itself assert len(altered) == 0 + new_state = State.QUEUED self._verify_dag_run_state(self.dag1, date, new_state) # type: ignore self._verify_task_instance_states_remain_default(dr) self._verify_dag_run_dates(self.dag1, date, new_state, middle_time) # type: ignore @pytest.mark.parametrize("completed_state", [State.SUCCESS, State.FAILED]) - def test_set_success_dag_run_to_success(self, completed_state): + def test_set_completed_dag_run_to_success(self, completed_state): date = self.execution_dates[0] dr = self._create_test_dag_run(completed_state, date) middle_time = timezone.utcnow() self._set_default_task_instance_states(dr) + old_end_date = dr.end_date altered = set_dag_run_state_to_success(dag=self.dag1, run_id=dr.run_id, commit=True) @@ -631,13 +651,14 @@ def test_set_success_dag_run_to_success(self, completed_state): assert len(altered) == expected self._verify_dag_run_state(self.dag1, date, State.SUCCESS) self._verify_task_instance_states(self.dag1, date, State.SUCCESS) - self._verify_dag_run_dates(self.dag1, date, State.SUCCESS, middle_time) + self._verify_dag_run_dates(self.dag1, date, State.SUCCESS, middle_time, old_end_date) @pytest.mark.parametrize("completed_state", [State.SUCCESS, State.FAILED]) def test_set_completed_dag_run_to_failed(self, completed_state): date = self.execution_dates[0] dr = self._create_test_dag_run(completed_state, date) middle_time = timezone.utcnow() + old_end_date = dr.end_date self._set_default_task_instance_states(dr) altered = set_dag_run_state_to_failed(dag=self.dag1, run_id=dr.run_id, commit=True) @@ -646,7 +667,7 @@ def test_set_completed_dag_run_to_failed(self, completed_state): assert len(altered) == expected self._verify_dag_run_state(self.dag1, date, State.FAILED) assert dr.get_task_instance("run_after_loop").state == State.FAILED - self._verify_dag_run_dates(self.dag1, date, State.FAILED, middle_time) + self._verify_dag_run_dates(self.dag1, date, State.FAILED, middle_time, old_end_date) @pytest.mark.parametrize( "dag_run_alter_function,new_state", diff --git a/tests/models/test_cleartasks.py b/tests/models/test_cleartasks.py index ed0232926aede3..bce9dc4668a1d4 100644 --- a/tests/models/test_cleartasks.py +++ b/tests/models/test_cleartasks.py @@ -210,7 +210,10 @@ def test_clear_task_instances_on_running_dr(self, state, dag_maker): session.refresh(dr) assert dr.state == state - assert dr.start_date + if state == DagRunState.QUEUED: + assert dr.start_date is None + if state == DagRunState.RUNNING: + assert dr.start_date assert dr.last_scheduling_decision == DEFAULT_DATE @pytest.mark.parametrize( diff --git a/tests/providers/amazon/aws/executors/ecs/test_ecs_executor.py b/tests/providers/amazon/aws/executors/ecs/test_ecs_executor.py index 04e777455572d5..8766659c05cbb3 100644 --- a/tests/providers/amazon/aws/executors/ecs/test_ecs_executor.py +++ b/tests/providers/amazon/aws/executors/ecs/test_ecs_executor.py @@ -34,6 +34,7 @@ from airflow.exceptions import AirflowException from airflow.executors.base_executor import BaseExecutor +from airflow.models.taskinstancekey import TaskInstanceKey from airflow.providers.amazon.aws.executors.ecs import ecs_executor, ecs_executor_config from airflow.providers.amazon.aws.executors.ecs.boto_schema import BotoTaskSchema from airflow.providers.amazon.aws.executors.ecs.ecs_executor import ( @@ -1156,3 +1157,220 @@ def test_providing_no_capacity_provider_no_lunch_type_no_cluster_default(self, m task_kwargs = ecs_executor_config.build_task_kwargs() assert task_kwargs["launchType"] == "FARGATE" + + @pytest.mark.parametrize( + "run_task_kwargs, exec_config, expected_result", + [ + # No input run_task_kwargs or executor overrides + ( + {}, + {}, + { + "taskDefinition": "some-task-def", + "launchType": "FARGATE", + "cluster": "some-cluster", + "platformVersion": "LATEST", + "count": 1, + "overrides": { + "containerOverrides": [ + { + "command": ["command"], + "name": "container-name", + "environment": [{"name": "AIRFLOW_IS_EXECUTOR_CONTAINER", "value": "true"}], + } + ] + }, + "networkConfiguration": { + "awsvpcConfiguration": { + "subnets": ["sub1", "sub2"], + "securityGroups": ["sg1", "sg2"], + "assignPublicIp": "DISABLED", + } + }, + }, + ), + # run_task_kwargs provided, not exec_config + ( + { + "startedBy": "Banana", + "tags": [{"key": "FOO", "value": "BAR"}], + "overrides": { + "containerOverrides": [ + { + "name": "container-name", + "memory": 500, + "cpu": 10, + "environment": [{"name": "X", "value": "Y"}], + } + ] + }, + }, + {}, + { + "startedBy": "Banana", + "tags": [{"key": "FOO", "value": "BAR"}], + "taskDefinition": "some-task-def", + "launchType": "FARGATE", + "cluster": "some-cluster", + "platformVersion": "LATEST", + "count": 1, + "overrides": { + "containerOverrides": [ + { + "memory": 500, + "cpu": 10, + "command": ["command"], + "name": "container-name", + "environment": [ + {"name": "X", "value": "Y"}, + # Added by the ecs executor + {"name": "AIRFLOW_IS_EXECUTOR_CONTAINER", "value": "true"}, + ], + } + ] + }, + # Added by the ecs executor + "networkConfiguration": { + "awsvpcConfiguration": { + "subnets": ["sub1", "sub2"], + "securityGroups": ["sg1", "sg2"], + "assignPublicIp": "DISABLED", + } + }, + }, + ), + # exec_config provided, no run_task_kwargs + ( + {}, + { + "startedBy": "Banana", + "tags": [{"key": "FOO", "value": "BAR"}], + "overrides": { + "containerOverrides": [ + { + "name": "container-name", + "memory": 500, + "cpu": 10, + "environment": [{"name": "X", "value": "Y"}], + } + ] + }, + }, + { + "startedBy": "Banana", + "tags": [{"key": "FOO", "value": "BAR"}], + "taskDefinition": "some-task-def", + "launchType": "FARGATE", + "cluster": "some-cluster", + "platformVersion": "LATEST", + "count": 1, + "overrides": { + "containerOverrides": [ + { + "memory": 500, + "cpu": 10, + "command": ["command"], + "name": "container-name", + "environment": [ + {"name": "X", "value": "Y"}, + # Added by the ecs executor + {"name": "AIRFLOW_IS_EXECUTOR_CONTAINER", "value": "true"}, + ], + } + ] + }, + # Added by the ecs executor + "networkConfiguration": { + "awsvpcConfiguration": { + "subnets": ["sub1", "sub2"], + "securityGroups": ["sg1", "sg2"], + "assignPublicIp": "DISABLED", + } + }, + }, + ), + # Both run_task_kwargs and executor_config provided. The latter should override the former, + # following a recursive python dict update strategy + ( + { + "startedBy": "Banana", + "tags": [{"key": "FOO", "value": "BAR"}], + "taskDefinition": "foobar", + "overrides": { + "containerOverrides": [ + { + "name": "container-name", + "memory": 500, + "cpu": 10, + "environment": [{"name": "X", "value": "Y"}], + } + ] + }, + }, + { + "startedBy": "Fish", + "tags": [{"key": "X", "value": "Y"}, {"key": "W", "value": "Z"}], + "overrides": { + "containerOverrides": [ + { + "name": "container-name", + "memory": 300, + "environment": [{"name": "W", "value": "Z"}], + } + ] + }, + }, + { + # tags and startedBy are overridden by exec_config + "startedBy": "Fish", + # List types overwrite entirely, as python dict update would do + "tags": [{"key": "X", "value": "Y"}, {"key": "W", "value": "Z"}], + # taskDefinition remains since it is not a list type and not overridden by exec config + "taskDefinition": "foobar", + "launchType": "FARGATE", + "cluster": "some-cluster", + "platformVersion": "LATEST", + "count": 1, + "overrides": { + "containerOverrides": [ + { + "memory": 300, + # cpu is not present because it was missing from the container overrides in + # the exec_config + "command": ["command"], + "name": "container-name", + "environment": [ + # Overridden list type + {"name": "W", "value": "Z"}, # Only new env vars present, overwritten + # Added by the ecs executor + {"name": "AIRFLOW_IS_EXECUTOR_CONTAINER", "value": "true"}, + ], + } + ] + }, + # Added by the ecs executor + "networkConfiguration": { + "awsvpcConfiguration": { + "subnets": ["sub1", "sub2"], + "securityGroups": ["sg1", "sg2"], + "assignPublicIp": "DISABLED", + } + }, + }, + ), + ], + ) + def test_run_task_kwargs_exec_config_overrides( + self, set_env_vars, run_task_kwargs, exec_config, expected_result + ): + run_task_kwargs_env_key = f"AIRFLOW__{CONFIG_GROUP_NAME}__{AllEcsConfigKeys.RUN_TASK_KWARGS}".upper() + os.environ[run_task_kwargs_env_key] = json.dumps(run_task_kwargs) + + mock_ti_key = mock.Mock(spec=TaskInstanceKey) + command = ["command"] + + executor = AwsEcsExecutor() + + final_run_task_kwargs = executor._run_task_kwargs(mock_ti_key, command, "queue", exec_config) + + assert final_run_task_kwargs == expected_result From 4dbf1c718b429e25c1260e6a3d957f52fb57ce5c Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Sat, 10 Feb 2024 14:30:47 +0530 Subject: [PATCH 16/21] format issue fixed --- docs/apache-airflow/extra-packages-ref.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/extra-packages-ref.rst b/docs/apache-airflow/extra-packages-ref.rst index 31697495d349b4..23234f9b2fc449 100644 --- a/docs/apache-airflow/extra-packages-ref.rst +++ b/docs/apache-airflow/extra-packages-ref.rst @@ -291,7 +291,7 @@ Some of those enable Airflow to use executors to run tasks with them - other tha +---------------------+-----------------------------------------------------+-----------------------------------------------------------------+----------------------------------------------+ | singularity | ``pip install 'apache-airflow[singularity]'`` | Singularity container operator | | +---------------------+-----------------------------------------------------+-----------------------------------------------------------------+----------------------------------------------+ -| teradata | ``pip install 'apache-airflow[teradata]'`` | Teradata hooks and operators | | +| teradata | ``pip install 'apache-airflow[teradata]'`` | Teradata hooks and operators | | +---------------------+-----------------------------------------------------+-----------------------------------------------------------------+----------------------------------------------+ | trino | ``pip install 'apache-airflow[trino]'`` | All Trino related operators & hooks | | +---------------------+-----------------------------------------------------+-----------------------------------------------------------------+----------------------------------------------+ From f3bb655ba600642fe3fe2fe69a1e4b3c6fe0ed0c Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Tue, 13 Feb 2024 09:08:08 +0530 Subject: [PATCH 17/21] Renamed parameter --- tests/providers/teradata/transfers/test_teradata_to_teradata.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/providers/teradata/transfers/test_teradata_to_teradata.py b/tests/providers/teradata/transfers/test_teradata_to_teradata.py index 496ca1dad35b5c..380a717f8afcdd 100644 --- a/tests/providers/teradata/transfers/test_teradata_to_teradata.py +++ b/tests/providers/teradata/transfers/test_teradata_to_teradata.py @@ -29,7 +29,7 @@ def test_execute(self): dest_teradata_conn_id = "dest_teradata_conn_id" destination_table = "destination_table" source_teradata_conn_id = "source_teradata_conn_id" - sql = (r"""select DATE where DATE > {{ source_sql_params.ref_date }};""",) + sql = (r"""select DATE where DATE > {{ sql_params.ref_date }};""",) sql_params = {"ref_date": "2018-01-01"} rows_chunk = 5000 cursor_description = [ From bb0648d1d6bb7089eac71e4b912671481e424cad Mon Sep 17 00:00:00 2001 From: Satish Ch Date: Tue, 13 Feb 2024 09:11:31 +0530 Subject: [PATCH 18/21] reverted nonrelated change --- docs/apache-airflow-providers-microsoft-mssql/operators.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/apache-airflow-providers-microsoft-mssql/operators.rst b/docs/apache-airflow-providers-microsoft-mssql/operators.rst index e2491c2d117f90..0dd55172150c20 100644 --- a/docs/apache-airflow-providers-microsoft-mssql/operators.rst +++ b/docs/apache-airflow-providers-microsoft-mssql/operators.rst @@ -43,7 +43,7 @@ An example usage of the MsSqlOperator is as follows: :start-after: [START howto_operator_mssql] :end-before: [END howto_operator_mssql] -You can also use an external file to execute the SQL commands. External sql file must be at the same level as DAG.py file. +You can also use an external file to execute the SQL commands. Script folder must be at the same level as DAG.py file. This way you can easily maintain the SQL queries separated from the code. .. exampleinclude:: /../../tests/system/providers/microsoft/mssql/example_mssql.py @@ -54,7 +54,7 @@ This way you can easily maintain the SQL queries separated from the code. Your ``dags/create_table.sql`` should look like this: -.. code-block:: sql +.. code-block::sql -- create Users table CREATE TABLE Users ( From e11d169e325b2b96d8f75a383c90423a5f2ba7c0 Mon Sep 17 00:00:00 2001 From: SatishChGit Date: Wed, 14 Feb 2024 01:18:09 -0800 Subject: [PATCH 19/21] Squashed commit of the following: commit ade9ce39038dd41337575947da49e5361a10c953 Author: SatishChGit Date: Wed Feb 14 01:04:46 2024 -0800 Redesigned teradatatoteradataoperator by removing private method --- INSTALL | 4 +- .../transfers/teradata_to_teradata.py | 32 ++++--- .../12_airflow_dependencies_and_extras.rst | 4 +- dev/breeze/doc/images/output_build-docs.txt | 2 +- ...ement_generate-issue-content-providers.svg | 2 +- ...ement_generate-issue-content-providers.txt | 2 +- ...agement_prepare-provider-documentation.svg | 2 +- ...agement_prepare-provider-documentation.txt | 2 +- ...e-management_prepare-provider-packages.svg | 2 +- ...e-management_prepare-provider-packages.txt | 2 +- ...output_release-management_publish-docs.txt | 2 +- .../transfers/test_teradata_to_teradata.py | 94 ++++++++++++++----- 12 files changed, 103 insertions(+), 47 deletions(-) diff --git a/INSTALL b/INSTALL index 576e1056824029..382313ca4e8eb0 100644 --- a/INSTALL +++ b/INSTALL @@ -255,8 +255,8 @@ microsoft-mssql, microsoft-psrp, microsoft-winrm, mongo, mssql, mysql, neo4j, od openfaas, openlineage, opensearch, opsgenie, oracle, otel, pagerduty, pandas, papermill, password, pgvector, pinecone, pinot, postgres, presto, pydantic, qdrant, rabbitmq, redis, s3, s3fs, salesforce, samba, saml, segment, sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, -spark, sqlite, ssh, statsd, tableau, tabular, telegram, teradata, trino, vertica, virtualenv, weaviate, -webhdfs, winrm, yandex, zendesk +spark, sqlite, ssh, statsd, tableau, tabular, telegram, teradata, trino, vertica, virtualenv, +weaviate, webhdfs, winrm, yandex, zendesk # END REGULAR EXTRAS HERE diff --git a/airflow/providers/teradata/transfers/teradata_to_teradata.py b/airflow/providers/teradata/transfers/teradata_to_teradata.py index dad8913ee3b774..5c003a43b8eda6 100644 --- a/airflow/providers/teradata/transfers/teradata_to_teradata.py +++ b/airflow/providers/teradata/transfers/teradata_to_teradata.py @@ -17,6 +17,7 @@ # under the License. from __future__ import annotations +from functools import cached_property from typing import TYPE_CHECKING, Sequence from airflow.models import BaseOperator @@ -71,21 +72,30 @@ def __init__( self.sql_params = sql_params self.rows_chunk = rows_chunk - def _execute(self, src_hook, dest_hook, context) -> None: + @cached_property + def src_hook(self) -> TeradataHook: + return TeradataHook(teradata_conn_id=self.source_teradata_conn_id) + + @cached_property + def dest_hook(self) -> TeradataHook: + return TeradataHook(teradata_conn_id=self.dest_teradata_conn_id) + + def execute(self, context: Context) -> None: + src_hook = self.src_hook + dest_hook = self.dest_hook with src_hook.get_conn() as src_conn: cursor = src_conn.cursor() cursor.execute(self.sql, self.sql_params) target_fields = [field[0] for field in cursor.description] rows_total = 0 - for rows in iter(lambda: cursor.fetchmany(self.rows_chunk), []): - dest_hook.bulk_insert_rows( - self.destination_table, rows, target_fields=target_fields, commit_every=self.rows_chunk - ) - rows_total += len(rows) + if len(target_fields) != 0: + for rows in iter(lambda: cursor.fetchmany(self.rows_chunk), []): + dest_hook.bulk_insert_rows( + self.destination_table, + rows, + target_fields=target_fields, + commit_every=self.rows_chunk, + ) + rows_total += len(rows) self.log.info("Finished data transfer. Total number of rows transferred - %s", rows_total) cursor.close() - - def execute(self, context: Context) -> None: - src_hook = TeradataHook(teradata_conn_id=self.source_teradata_conn_id) - dest_hook = TeradataHook(teradata_conn_id=self.dest_teradata_conn_id) - self._execute(src_hook, dest_hook, context) diff --git a/contributing-docs/12_airflow_dependencies_and_extras.rst b/contributing-docs/12_airflow_dependencies_and_extras.rst index 925ac1e06bdd0c..489c8347150352 100644 --- a/contributing-docs/12_airflow_dependencies_and_extras.rst +++ b/contributing-docs/12_airflow_dependencies_and_extras.rst @@ -211,8 +211,8 @@ microsoft-mssql, microsoft-psrp, microsoft-winrm, mongo, mssql, mysql, neo4j, od openfaas, openlineage, opensearch, opsgenie, oracle, otel, pagerduty, pandas, papermill, password, pgvector, pinecone, pinot, postgres, presto, pydantic, qdrant, rabbitmq, redis, s3, s3fs, salesforce, samba, saml, segment, sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, -spark, sqlite, ssh, statsd, tableau, tabular, telegram, teradata, trino, vertica, virtualenv, weaviate, -webhdfs, winrm, yandex, zendesk +spark, sqlite, ssh, statsd, tableau, tabular, telegram, teradata, trino, vertica, virtualenv, +weaviate, webhdfs, winrm, yandex, zendesk .. END REGULAR EXTRAS HERE diff --git a/dev/breeze/doc/images/output_build-docs.txt b/dev/breeze/doc/images/output_build-docs.txt index 0fb92ded1ee565..40ddcc00954c38 100644 --- a/dev/breeze/doc/images/output_build-docs.txt +++ b/dev/breeze/doc/images/output_build-docs.txt @@ -1 +1 @@ -e19c33fee49a19faae0e3f7cea92a93e +40e06822e487b6d37b637cd9db881315 diff --git a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg index 57129c3e437e40..81c991dda14749 100644 --- a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg +++ b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.svg @@ -146,7 +146,7 @@ microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage |  opensearch | opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | qdrant | redis |    salesforce | samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau |     -tabular | telegram | trino | vertica | weaviate | yandex | zendesk]...                                                 +tabular | telegram | teradata | trino | vertica | weaviate | yandex | zendesk]...                                      Generates content for issue to test the release. diff --git a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt index 5024a687f502d3..a0763fc4aae127 100644 --- a/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt +++ b/dev/breeze/doc/images/output_release-management_generate-issue-content-providers.txt @@ -1 +1 @@ -d710025e44f0aade0a6ba788d2ac481b +54a463f51bb58e6159a7a8eb9bac3748 diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg index a51a74cf9ba6fe..1aa213332d46b1 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.svg @@ -182,7 +182,7 @@ microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage |  opensearch | opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | qdrant | redis |    salesforce | samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau |     -tabular | telegram | trino | vertica | weaviate | yandex | zendesk]...                                                 +tabular | telegram | teradata | trino | vertica | weaviate | yandex | zendesk]...                                      Prepare CHANGELOG, README and COMMITS information for providers. diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt index edba94be192097..104cfee713b74c 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-documentation.txt @@ -1 +1 @@ -5e8f0f4051dc0e7895a004c52b9cdb8a +ba0a7fc905c4d7e5f503917ebeb92cae diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-packages.svg b/dev/breeze/doc/images/output_release-management_prepare-provider-packages.svg index f80e635bfc6321..8d8415a2b80035 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-packages.svg +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-packages.svg @@ -167,7 +167,7 @@ microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql | neo4j | odbc | openai | openfaas | openlineage |  opensearch | opsgenie | oracle | pagerduty | papermill | pgvector | pinecone | postgres | presto | qdrant | redis |    salesforce | samba | segment | sendgrid | sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau |     -tabular | telegram | trino | vertica | weaviate | yandex | zendesk]...                                                 +tabular | telegram | teradata | trino | vertica | weaviate | yandex | zendesk]...                                      Prepare sdist/whl packages of Airflow Providers. diff --git a/dev/breeze/doc/images/output_release-management_prepare-provider-packages.txt b/dev/breeze/doc/images/output_release-management_prepare-provider-packages.txt index bd01748ec75d2e..54a07c40189fd4 100644 --- a/dev/breeze/doc/images/output_release-management_prepare-provider-packages.txt +++ b/dev/breeze/doc/images/output_release-management_prepare-provider-packages.txt @@ -1 +1 @@ -df4076b125bf24eeafb45a1fdc4c9c15 +c275700c380ceda02285216e708d6798 diff --git a/dev/breeze/doc/images/output_release-management_publish-docs.txt b/dev/breeze/doc/images/output_release-management_publish-docs.txt index 144a3c83850bcd..776b7f7fcabae8 100644 --- a/dev/breeze/doc/images/output_release-management_publish-docs.txt +++ b/dev/breeze/doc/images/output_release-management_publish-docs.txt @@ -1 +1 @@ -6cf227f633110d8437d036eb6f29dbd3 +c5ec5274613346fcc37d187485028ca7 diff --git a/tests/providers/teradata/transfers/test_teradata_to_teradata.py b/tests/providers/teradata/transfers/test_teradata_to_teradata.py index 380a717f8afcdd..8f20e1f571f125 100644 --- a/tests/providers/teradata/transfers/test_teradata_to_teradata.py +++ b/tests/providers/teradata/transfers/test_teradata_to_teradata.py @@ -19,19 +19,65 @@ from decimal import Decimal from unittest import mock -from unittest.mock import MagicMock +import pytest + +from airflow.providers.teradata.hooks.teradata import TeradataHook from airflow.providers.teradata.transfers.teradata_to_teradata import TeradataToTeradataOperator +@pytest.fixture +def mocked_src_hook(): + with mock.patch.object( + TeradataToTeradataOperator, "src_hook", spec=TeradataHook, name="TeradataSourceHook" + ) as m: + yield m + + +@pytest.fixture +def mocked_dest_hook(): + with mock.patch.object( + TeradataToTeradataOperator, "dest_hook", spec=TeradataHook, name="TeradataDestHook" + ) as m: + yield m + + class TestTeradataToTeradataTransfer: - def test_execute(self): - dest_teradata_conn_id = "dest_teradata_conn_id" - destination_table = "destination_table" - source_teradata_conn_id = "source_teradata_conn_id" - sql = (r"""select DATE where DATE > {{ sql_params.ref_date }};""",) - sql_params = {"ref_date": "2018-01-01"} - rows_chunk = 5000 + dest_teradata_conn_id = "dest_teradata_conn_id" + destination_table = "destination_table" + source_teradata_conn_id = "source_teradata_conn_id" + sql = (r"""select DATE where DATE > {{ sql_params.ref_date }} ;""",) + sql_params = {"ref_date": "2018-01-01"} + + def test_source_hook(self): + op = TeradataToTeradataOperator( + task_id="transfer_data", + dest_teradata_conn_id=self.dest_teradata_conn_id, + destination_table=self.destination_table, + source_teradata_conn_id=self.source_teradata_conn_id, + sql=self.sql, + sql_params=self.sql_params, + ) + hook = op.src_hook + assert hook + assert hook is op.src_hook + assert hook.teradata_conn_id == "source_teradata_conn_id" + + def test_destination_hook(self): + op = TeradataToTeradataOperator( + task_id="transfer_data", + dest_teradata_conn_id=self.dest_teradata_conn_id, + destination_table=self.destination_table, + source_teradata_conn_id=self.source_teradata_conn_id, + sql=self.sql, + sql_params=self.sql_params, + ) + hook = op.dest_hook + assert hook + assert hook is op.dest_hook + assert hook.teradata_conn_id == "dest_teradata_conn_id" + + def test_execution(self, mocked_src_hook, mocked_dest_hook): cursor_description = [ ["user_id", Decimal, None, 8, 10, 0, False], ["user_name", str, None, 60, None, None, True], @@ -39,33 +85,33 @@ def test_execute(self): cursor_rows = [[Decimal("1"), "User1"], [Decimal("2"), "User2"], [Decimal("3"), "User3"]] - mock_dest_hook = MagicMock() - mock_src_hook = MagicMock() - mock_src_conn = mock_src_hook.get_conn.return_value.__enter__.return_value + mock_src_conn = mocked_src_hook.get_conn.return_value.__enter__.return_value mock_cursor = mock_src_conn.cursor.return_value mock_cursor.description.__iter__.return_value = cursor_description mock_cursor.fetchmany.side_effect = [cursor_rows, []] + rows_chunk = 5000 - td_transfer_op = TeradataToTeradataOperator( + op = TeradataToTeradataOperator( task_id="transfer_data", - dest_teradata_conn_id=dest_teradata_conn_id, - destination_table=destination_table, - source_teradata_conn_id=source_teradata_conn_id, - sql=sql, - sql_params=sql_params, - rows_chunk=rows_chunk, + dest_teradata_conn_id=self.dest_teradata_conn_id, + destination_table=self.destination_table, + source_teradata_conn_id=self.source_teradata_conn_id, + sql=self.sql, + sql_params=self.sql_params, ) + op.execute({}) - td_transfer_op._execute(mock_src_hook, mock_dest_hook, None) - - assert mock_src_hook.get_conn.called + assert mocked_src_hook.get_conn.called assert mock_src_conn.cursor.called - mock_cursor.execute.assert_called_once_with(sql, sql_params) + mock_cursor.execute.assert_called_once_with(self.sql, self.sql_params) calls = [ mock.call(rows_chunk), ] mock_cursor.fetchmany.assert_has_calls(calls) - mock_dest_hook.bulk_insert_rows.assert_called_once_with( - destination_table, cursor_rows, commit_every=rows_chunk, target_fields=["user_id", "user_name"] + mocked_dest_hook.bulk_insert_rows.assert_called_once_with( + self.destination_table, + cursor_rows, + commit_every=rows_chunk, + target_fields=["user_id", "user_name"], ) From dfe996703a06802787e708f96f228349237f5515 Mon Sep 17 00:00:00 2001 From: SatishChGit Date: Thu, 15 Feb 2024 13:12:34 +0530 Subject: [PATCH 20/21] Update airflow/providers/teradata/CHANGELOG.rst Co-authored-by: Elad Kalif <45845474+eladkal@users.noreply.github.com> --- airflow/providers/teradata/CHANGELOG.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/teradata/CHANGELOG.rst b/airflow/providers/teradata/CHANGELOG.rst index 46857d51a98134..cb46e0d75471e3 100644 --- a/airflow/providers/teradata/CHANGELOG.rst +++ b/airflow/providers/teradata/CHANGELOG.rst @@ -28,4 +28,4 @@ Changelog 1.0.0 ..... -Initial version of the provider. +``Initial version of the provider. (#36953)`` From 18475fd3c73cb6e67ca19eb8f8a93933e9af623d Mon Sep 17 00:00:00 2001 From: SatishChGit Date: Thu, 15 Feb 2024 20:29:47 +0530 Subject: [PATCH 21/21] Update docs/apache-airflow-providers-teradata/operators/teradata.rst Co-authored-by: Josh Fell <48934154+josh-fell@users.noreply.github.com> --- docs/apache-airflow-providers-teradata/operators/teradata.rst | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/apache-airflow-providers-teradata/operators/teradata.rst b/docs/apache-airflow-providers-teradata/operators/teradata.rst index 2e98341a4cc121..e894d37ade66c8 100644 --- a/docs/apache-airflow-providers-teradata/operators/teradata.rst +++ b/docs/apache-airflow-providers-teradata/operators/teradata.rst @@ -35,6 +35,7 @@ An example usage of the TeradataOperator is as follows: .. exampleinclude:: /../../tests/system/providers/teradata/example_teradata.py :language: python + :dedent: 4 :start-after: [START teradata_operator_howto_guide_create_table] :end-before: [END teradata_operator_howto_guide_create_table]