diff --git a/.github/boring-cyborg.yml b/.github/boring-cyborg.yml index fd98dc5767e09..2ea0914434abf 100644 --- a/.github/boring-cyborg.yml +++ b/.github/boring-cyborg.yml @@ -118,11 +118,6 @@ labelPRBasedOnFilePath: - tests/providers/apache/spark/**/* - tests/system/providers/apache/spark/**/* - provider:apache-sqoop: - - airflow/providers/apache/sqoop/**/* - - docs/apache-airflow-providers-apache-sqoop/**/* - - tests/providers/apache/sqoop/**/* - provider:apprise: - airflow/providers/apprise/**/* - docs/apache-airflow-providers-apprise/**/* @@ -391,12 +386,6 @@ labelPRBasedOnFilePath: - tests/providers/pinecone/**/* - tests/system/providers/pinecone/**/* - provider:plexus: - - airflow/providers/plexus/**/* - - docs/apache-airflow-providers-plexus/**/* - - tests/providers/plexus/**/* - - tests/system/providers/plexus/**/* - provider:postgres: - airflow/providers/postgres/**/* - docs/apache-airflow-providers-postgres/**/* diff --git a/airflow/contrib/hooks/__init__.py b/airflow/contrib/hooks/__init__.py index b2c7612989489..63306a5ec3b42 100644 --- a/airflow/contrib/hooks/__init__.py +++ b/airflow/contrib/hooks/__init__.py @@ -311,9 +311,6 @@ "spark_submit_hook": { "SparkSubmitHook": "airflow.providers.apache.spark.hooks.spark_submit.SparkSubmitHook", }, - "sqoop_hook": { - "SqoopHook": "airflow.providers.apache.sqoop.hooks.sqoop.SqoopHook", - }, "ssh_hook": { "SSHHook": "airflow.providers.ssh.hooks.ssh.SSHHook", }, diff --git a/airflow/contrib/operators/__init__.py b/airflow/contrib/operators/__init__.py index faf1899606a3b..0bfa8e6724614 100644 --- a/airflow/contrib/operators/__init__.py +++ b/airflow/contrib/operators/__init__.py @@ -1099,9 +1099,6 @@ "airflow.providers.google.cloud.transfers.sql_to_gcs.BaseSQLToGCSOperator" ), }, - "sqoop_operator": { - "SqoopOperator": "airflow.providers.apache.sqoop.operators.sqoop.SqoopOperator", - }, "ssh_operator": { "SSHOperator": "airflow.providers.ssh.operators.ssh.SSHOperator", }, diff --git a/airflow/providers/apache/sqoop/.latest-doc-only-change.txt b/airflow/providers/apache/sqoop/.latest-doc-only-change.txt deleted file mode 100644 index 36cd4c153e278..0000000000000 --- a/airflow/providers/apache/sqoop/.latest-doc-only-change.txt +++ /dev/null @@ -1 +0,0 @@ -99534e47f330ce0efb96402629dda5b2a4f16e8f diff --git a/airflow/providers/apache/sqoop/CHANGELOG.rst b/airflow/providers/apache/sqoop/CHANGELOG.rst deleted file mode 100644 index 87dde5c591ffb..0000000000000 --- a/airflow/providers/apache/sqoop/CHANGELOG.rst +++ /dev/null @@ -1,300 +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. - - -.. 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-apache-sqoop`` - - -Changelog ---------- - - -4.2.1 -..... - -.. note:: - - Mark apache.sqoop provider as removed according to `consensus on the dev@airflow.apache.org `_ - - -Misc -~~~~ - -* ``Schedule plexus and apache.sqoop providers for removal (#36208)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - -4.2.0 -..... - -.. note:: - This release of provider is only available for Airflow 2.6+ as explained in the - `Apache Airflow providers support policy `_. - -Misc -~~~~ - -* ``Bump minimum Airflow version in providers to Airflow 2.6.0 (#36017)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Fix and reapply templates for provider documentation (#35686)`` - * ``Prepare docs 3rd wave of Providers October 2023 - FIX (#35233)`` - * ``Prepare docs 2nd wave of Providers November 2023 (#35836)`` - * ``Use reproducible builds for provider packages (#35693)`` - * ``Prepare docs 1st wave of Providers November 2023 (#35537)`` - * ``Prepare docs 3rd wave of Providers October 2023 (#35187)`` - * ``Pre-upgrade 'ruff==0.0.292' changes in providers (#35053)`` - * ``Upgrade pre-commits (#35033)`` - -4.1.0 -..... - -.. note:: - This release of provider is only available for Airflow 2.5+ as explained in the - `Apache Airflow providers support policy `_. - -Misc -~~~~ - -* ``Bump min airflow version of providers (#34728)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Prepare docs for 09 2023 - 1st wave of Providers (#34201)`` - * ``Prepare docs for Aug 2023 3rd wave of Providers (#33730)`` - * ``D401 Support - Providers: Airbyte to Atlassian (Inclusive) (#33354)`` - -4.0.0 -..... - -Breaking changes -~~~~~~~~~~~~~~~~ - -The ``extra_import_options`` parameter in the ``import_table`` & ``import_query`` methods -and the ``extra_export_options`` in the ``export_table`` methods of the ``SqoopHook`` -are no longer accepted as arguments for those methods. These should instead be passed -as ``extra_options`` while initializing the Hook or via ``extra_options`` parameter to the -operator which instantiates the hook with those given ``extra_options`` dictionary. - -* ``Validate SqoopHook connection string and disable extra options from public hook methods (#33039)`` - -.. Review and move the new changes to one of the sections above: - * ``Validate connection host field for Sqoop connection (#32968)`` - * ``Prepare docs for July 2023 wave of Providers (RC2) (#32381)`` - * ``Remove spurious headers for provider changelogs (#32373)`` - * ``Prepare docs for July 2023 wave of Providers (#32298)`` - * ``Improve provider documentation and README structure (#32125)`` - -3.2.1 -..... - -.. note:: - This release dropped support for Python 3.7 - - -Misc -~~~~ - -* ``Add note about dropping Python 3.7 for providers (#32015)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Apache provider docstring improvements (#31730)`` - * ``Add D400 pydocstyle check - Apache providers only (#31424)`` - -3.2.0 -..... - -.. note:: - This release of provider is only available for Airflow 2.4+ as explained in the - `Apache Airflow providers support policy `_. - -Misc -~~~~ - -* ``Bump minimum Airflow version in providers (#30917)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Add full automation for min Airflow version for providers (#30994)`` - * ``Add mechanism to suspend providers (#30422)`` - * ``Use '__version__' in providers not 'version' (#31393)`` - * ``Fixing circular import error in providers caused by airflow version check (#31379)`` - * ``Prepare docs for May 2023 wave of Providers (#31252)`` - -3.1.1 -..... - -Bug Fixes -~~~~~~~~~ - -* ``Move libjars parameter in Sqoop Hook to Hook parameter (#29500)`` - -3.1.0 -..... - -.. note:: - This release of provider is only available for Airflow 2.3+ as explained in the - `Apache Airflow providers support policy `_. - -Misc -~~~~ - -* ``Move min airflow version to 2.3.0 for all providers (#27196)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Add documentation for July 2022 Provider's release (#25030)`` - * ``Enable string normalization in python formatting - providers (#27205)`` - * ``Update docs for September Provider's release (#26731)`` - * ``Apply PEP-563 (Postponed Evaluation of Annotations) to non-core airflow (#26289)`` - * ``Prepare docs for new providers release (August 2022) (#25618)`` - * ``Move provider dependencies to inside provider folders (#24672)`` - * ``Remove 'hook-class-names' from provider.yaml (#24702)`` - -3.0.0 -..... - -Breaking changes -~~~~~~~~~~~~~~~~ - -.. note:: - This release of provider is only available for Airflow 2.2+ as explained in the - `Apache Airflow providers support policy `_. - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Add explanatory note for contributors about updating Changelog (#24229)`` - * ``Prepare docs for May 2022 provider's release (#24231)`` - * ``Update package description to remove double min-airflow specification (#24292)`` - -2.1.3 -..... - -Bug Fixes -~~~~~~~~~ - -* ``Fix mistakenly added install_requires for all providers (#22382)`` - -2.1.2 -..... - -Misc -~~~~~ - -* ``Add Trove classifiers in PyPI (Framework :: Apache Airflow :: Provider)`` - -2.1.1 -..... - -Misc -~~~~ - -* ``Support for Python 3.10`` -* ``add how-to guide for sqoop operator (#21424)`` - - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - -2.1.0 -..... - -Features -~~~~~~~~ - -* ``Add more SQL template fields renderers (#21237)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Fix K8S changelog to be PyPI-compatible (#20614)`` - * ``Fix template_fields type to have MyPy friendly Sequence type (#20571)`` - * ``Fix MyPy errors in Apache Providers (#20422)`` - * ``Fix MyPy Errors for Apache Sqoop provider. (#20304)`` - * ``Remove ':type' lines now sphinx-autoapi supports typehints (#20951)`` - * ``Update documentation for provider December 2021 release (#20523)`` - * ``Add documentation for January 2021 providers release (#21257)`` - -2.0.2 -..... - -Misc -~~~~ - -* ``Optimise connection importing for Airflow 2.2.0`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Update description about the new ''connection-types'' provider meta-data (#17767)`` - * ``Import Hooks lazily individually in providers manager (#17682)`` - -2.0.1 -..... - -Bug Fixes -~~~~~~~~~ - -* ``Fix Minor Bugs in Apache Sqoop Hook and Operator (#16350)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Removes pylint from our toolchain (#16682)`` - * ``Prepare documentation for July release of providers. (#17015)`` - * ``Fixed wrongly escaped characters in amazon's changelog (#17020)`` - -2.0.0 -..... - -Breaking changes -~~~~~~~~~~~~~~~~ - -* ``Auto-apply apply_default decorator (#15667)`` - -.. warning:: Due to apply_default decorator removal, this version of the provider requires Airflow 2.1.0+. - If your Airflow version is < 2.1.0, and you want to install this provider version, first upgrade - Airflow to at least version 2.1.0. Otherwise your Airflow package version will be upgraded - automatically and you will have to manually run ``airflow upgrade db`` to complete the migration. - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Fixes failing static checks after recent pre-commit upgrade (#16183)`` - * ``Adds interactivity when generating provider documentation. (#15518)`` - * ``Prepares provider release after PIP 21 compatibility (#15576)`` - * ``Make Airflow code Pylint 2.8 compatible (#15534)`` - * ``Remove Backport Providers (#14886)`` - * ``Update documentation for broken package releases (#14734)`` - * ``Updated documentation for June 2021 provider release (#16294)`` - * ``More documentation update for June providers release (#16405)`` - * ``Synchronizes updated changelog after buggfix release (#16464)`` - -1.0.1 -..... - -Updated documentation and readme files. - -1.0.0 -..... - -Initial version of the provider. diff --git a/airflow/providers/apache/sqoop/__init__.py b/airflow/providers/apache/sqoop/__init__.py deleted file mode 100644 index d263980405c34..0000000000000 --- a/airflow/providers/apache/sqoop/__init__.py +++ /dev/null @@ -1,42 +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. -# -# NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE -# OVERWRITTEN WHEN PREPARING DOCUMENTATION FOR THE PACKAGES. -# -# IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE -# `PROVIDER__INIT__PY_TEMPLATE.py.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY -# -from __future__ import annotations - -import packaging.version - -__all__ = ["__version__"] - -__version__ = "4.2.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( - "2.6.0" -): - raise RuntimeError( - f"The package `apache-airflow-providers-apache-sqoop:{__version__}` needs Apache Airflow 2.6.0+" - ) diff --git a/airflow/providers/apache/sqoop/hooks/__init__.py b/airflow/providers/apache/sqoop/hooks/__init__.py deleted file mode 100644 index 217e5db960782..0000000000000 --- a/airflow/providers/apache/sqoop/hooks/__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/apache/sqoop/hooks/sqoop.py b/airflow/providers/apache/sqoop/hooks/sqoop.py deleted file mode 100644 index 872c4a0462b70..0000000000000 --- a/airflow/providers/apache/sqoop/hooks/sqoop.py +++ /dev/null @@ -1,411 +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. -"""This module contains a sqoop 1.x hook.""" -from __future__ import annotations - -import subprocess -from copy import deepcopy -from typing import Any - -from airflow.exceptions import AirflowException -from airflow.hooks.base import BaseHook - - -class SqoopHook(BaseHook): - """Wrapper around the sqoop 1 binary. - - To be able to use the hook, it is required that "sqoop" is in the PATH. - - Additional arguments that can be passed via the 'extra' JSON field of the - sqoop connection: - - * ``job_tracker``: Job tracker local|jobtracker:port. - * ``namenode``: Namenode. - * ``files``: Comma separated files to be copied to the map reduce cluster. - * ``archives``: Comma separated archives to be unarchived on the compute - machines. - * ``password_file``: Path to file containing the password. - - :param conn_id: Reference to the sqoop connection. - :param verbose: Set sqoop to verbose. - :param num_mappers: Number of map tasks to import in parallel. - :param properties: Properties to set via the -D argument - :param libjars: Optional Comma separated jar files to include in the classpath. - :param extra_options: Extra import/export options to pass as dict. - If a key doesn't have a value, just pass an empty string to it. - Don't include prefix of -- for sqoop options. - """ - - conn_name_attr = "conn_id" - default_conn_name = "sqoop_default" - conn_type = "sqoop" - hook_name = "Sqoop" - - def __init__( - self, - conn_id: str = default_conn_name, - verbose: bool = False, - num_mappers: int | None = None, - hcatalog_database: str | None = None, - hcatalog_table: str | None = None, - properties: dict[str, Any] | None = None, - libjars: str | None = None, - extra_options: dict[str, Any] | None = None, - ) -> None: - # No mutable types in the default parameters - super().__init__() - self.conn = self.get_connection(conn_id) - connection_parameters = self.conn.extra_dejson - self.job_tracker = connection_parameters.get("job_tracker", None) - self.namenode = connection_parameters.get("namenode", None) - self.libjars = libjars - self.files = connection_parameters.get("files", None) - self.archives = connection_parameters.get("archives", None) - self.password_file = connection_parameters.get("password_file", None) - self.hcatalog_database = hcatalog_database - self.hcatalog_table = hcatalog_table - self.verbose = verbose - self.num_mappers = num_mappers - self.properties = properties or {} - self.sub_process_pid: int - self._extra_options = extra_options - self.log.info("Using connection to: %s:%s/%s", self.conn.host, self.conn.port, self.conn.schema) - - def get_conn(self) -> Any: - return self.conn - - def cmd_mask_password(self, cmd_orig: list[str]) -> list[str]: - """Mask command password for safety.""" - cmd = deepcopy(cmd_orig) - try: - password_index = cmd.index("--password") - cmd[password_index + 1] = "MASKED" - except ValueError: - self.log.debug("No password in sqoop cmd") - return cmd - - def popen(self, cmd: list[str], **kwargs: Any) -> None: - """Remote Popen. - - :param cmd: command to remotely execute - :param kwargs: extra arguments to Popen (see subprocess.Popen) - :return: handle to subprocess - """ - masked_cmd = " ".join(self.cmd_mask_password(cmd)) - self.log.info("Executing command: %s", masked_cmd) - with subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, **kwargs) as sub_process: - self.sub_process_pid = sub_process.pid - for line in iter(sub_process.stdout): # type: ignore - self.log.info(line.strip()) - sub_process.wait() - self.log.info("Command exited with return code %s", sub_process.returncode) - if sub_process.returncode: - raise AirflowException(f"Sqoop command failed: {masked_cmd}") - - def _prepare_command(self, export: bool = False) -> list[str]: - sqoop_cmd_type = "export" if export else "import" - connection_cmd = ["sqoop", sqoop_cmd_type] - - for key, value in self.properties.items(): - connection_cmd += ["-D", f"{key}={value}"] - - if self.namenode: - connection_cmd += ["-fs", self.namenode] - if self.job_tracker: - connection_cmd += ["-jt", self.job_tracker] - if self.libjars: - connection_cmd += ["-libjars", self.libjars] - if self.files: - connection_cmd += ["-files", self.files] - if self.archives: - connection_cmd += ["-archives", self.archives] - if self.conn.login: - connection_cmd += ["--username", self.conn.login] - if self.conn.password: - connection_cmd += ["--password", self.conn.password] - if self.password_file: - connection_cmd += ["--password-file", self.password_file] - if self.verbose: - connection_cmd += ["--verbose"] - if self.num_mappers: - connection_cmd += ["--num-mappers", str(self.num_mappers)] - if self.hcatalog_database: - connection_cmd += ["--hcatalog-database", self.hcatalog_database] - if self.hcatalog_table: - connection_cmd += ["--hcatalog-table", self.hcatalog_table] - - connect_str = self.conn.host - if self.conn.port: - connect_str += f":{self.conn.port}" - if self.conn.schema: - self.log.info("CONNECTION TYPE %s", self.conn.conn_type) - if self.conn.conn_type != "mssql": - connect_str += f"/{self.conn.schema}" - else: - connect_str += f";databaseName={self.conn.schema}" - if "?" in connect_str: - raise ValueError("The sqoop connection string should not contain a '?' character") - connection_cmd += ["--connect", connect_str] - - return connection_cmd - - @staticmethod - def _get_export_format_argument(file_type: str = "text") -> list[str]: - if file_type == "avro": - return ["--as-avrodatafile"] - elif file_type == "sequence": - return ["--as-sequencefile"] - elif file_type == "parquet": - return ["--as-parquetfile"] - elif file_type == "text": - return ["--as-textfile"] - else: - raise AirflowException("Argument file_type should be 'avro', 'sequence', 'parquet' or 'text'.") - - def _import_cmd( - self, - target_dir: str | None, - append: bool, - file_type: str, - split_by: str | None, - direct: bool | None, - driver: Any, - ) -> list[str]: - cmd = self._prepare_command(export=False) - - if target_dir: - cmd += ["--target-dir", target_dir] - - if append: - cmd += ["--append"] - - cmd += self._get_export_format_argument(file_type) - - if split_by: - cmd += ["--split-by", split_by] - - if direct: - cmd += ["--direct"] - - if driver: - cmd += ["--driver", driver] - - if self._extra_options: - for key, value in self._extra_options.items(): - cmd += [f"--{key}"] - if value: - cmd += [str(value)] - - return cmd - - def import_table( - self, - table: str, - target_dir: str | None = None, - append: bool = False, - file_type: str = "text", - columns: str | None = None, - split_by: str | None = None, - where: str | None = None, - direct: bool = False, - driver: Any = None, - schema: str | None = None, - ) -> Any: - """Import table from remote location to target dir. - - Arguments are copies of direct sqoop command line arguments. - - :param table: Table to read - :param schema: Schema name - :param target_dir: HDFS destination dir - :param append: Append data to an existing dataset in HDFS - :param file_type: "avro", "sequence", "text" or "parquet". - Imports data to into the specified format. Defaults to text. - :param columns: Columns to import from table - :param split_by: Column of the table used to split work units - :param where: WHERE clause to use during import - :param direct: Use direct connector if exists for the database - :param driver: Manually specify JDBC driver class to use - """ - cmd = self._import_cmd(target_dir, append, file_type, split_by, direct, driver) - - cmd += ["--table", table] - - if columns: - cmd += ["--columns", columns] - if where: - cmd += ["--where", where] - if schema: - cmd += ["--", "--schema", schema] - - self.popen(cmd) - - def import_query( - self, - query: str, - target_dir: str | None = None, - append: bool = False, - file_type: str = "text", - split_by: str | None = None, - direct: bool | None = None, - driver: Any | None = None, - ) -> Any: - """Import a specific query from the rdbms to hdfs. - - :param query: Free format query to run - :param target_dir: HDFS destination dir - :param append: Append data to an existing dataset in HDFS - :param file_type: "avro", "sequence", "text" or "parquet" - Imports data to hdfs into the specified format. Defaults to text. - :param split_by: Column of the table used to split work units - :param direct: Use direct import fast path - :param driver: Manually specify JDBC driver class to use - """ - cmd = self._import_cmd(target_dir, append, file_type, split_by, direct, driver) - cmd += ["--query", query] - - self.popen(cmd) - - def _export_cmd( - self, - table: str, - export_dir: str | None = None, - input_null_string: str | None = None, - input_null_non_string: str | None = None, - staging_table: str | None = None, - clear_staging_table: bool = False, - enclosed_by: str | None = None, - escaped_by: str | None = None, - input_fields_terminated_by: str | None = None, - input_lines_terminated_by: str | None = None, - input_optionally_enclosed_by: str | None = None, - batch: bool = False, - relaxed_isolation: bool = False, - schema: str | None = None, - ) -> list[str]: - cmd = self._prepare_command(export=True) - - if input_null_string: - cmd += ["--input-null-string", input_null_string] - - if input_null_non_string: - cmd += ["--input-null-non-string", input_null_non_string] - - if staging_table: - cmd += ["--staging-table", staging_table] - - if clear_staging_table: - cmd += ["--clear-staging-table"] - - if enclosed_by: - cmd += ["--enclosed-by", enclosed_by] - - if escaped_by: - cmd += ["--escaped-by", escaped_by] - - if input_fields_terminated_by: - cmd += ["--input-fields-terminated-by", input_fields_terminated_by] - - if input_lines_terminated_by: - cmd += ["--input-lines-terminated-by", input_lines_terminated_by] - - if input_optionally_enclosed_by: - cmd += ["--input-optionally-enclosed-by", input_optionally_enclosed_by] - - if batch: - cmd += ["--batch"] - - if relaxed_isolation: - cmd += ["--relaxed-isolation"] - - if export_dir: - cmd += ["--export-dir", export_dir] - - if self._extra_options: - for key, value in self._extra_options.items(): - cmd += [f"--{key}"] - if value: - cmd += [str(value)] - - # The required option - cmd += ["--table", table] - - if schema: - cmd += ["--", "--schema", schema] - - return cmd - - def export_table( - self, - table: str, - export_dir: str | None = None, - input_null_string: str | None = None, - input_null_non_string: str | None = None, - staging_table: str | None = None, - clear_staging_table: bool = False, - enclosed_by: str | None = None, - escaped_by: str | None = None, - input_fields_terminated_by: str | None = None, - input_lines_terminated_by: str | None = None, - input_optionally_enclosed_by: str | None = None, - batch: bool = False, - relaxed_isolation: bool = False, - schema: str | None = None, - ) -> None: - """Export Hive table to remote location. - - Arguments are copies of direct Sqoop command line Arguments - - :param table: Table remote destination - :param schema: Schema name - :param export_dir: Hive table to export - :param input_null_string: The string to be interpreted as null for - string columns - :param input_null_non_string: The string to be interpreted as null - for non-string columns - :param staging_table: The table in which data will be staged before - being inserted into the destination table - :param clear_staging_table: Indicate that any data present in the - staging table can be deleted - :param enclosed_by: Sets a required field enclosing character - :param escaped_by: Sets the escape character - :param input_fields_terminated_by: Sets the field separator character - :param input_lines_terminated_by: Sets the end-of-line character - :param input_optionally_enclosed_by: Sets a field enclosing character - :param batch: Use batch mode for underlying statement execution - :param relaxed_isolation: Transaction isolation to read uncommitted - for the mappers - """ - cmd = self._export_cmd( - table, - export_dir, - input_null_string, - input_null_non_string, - staging_table, - clear_staging_table, - enclosed_by, - escaped_by, - input_fields_terminated_by, - input_lines_terminated_by, - input_optionally_enclosed_by, - batch, - relaxed_isolation, - schema, - ) - - self.popen(cmd) diff --git a/airflow/providers/apache/sqoop/operators/__init__.py b/airflow/providers/apache/sqoop/operators/__init__.py deleted file mode 100644 index 217e5db960782..0000000000000 --- a/airflow/providers/apache/sqoop/operators/__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/apache/sqoop/operators/sqoop.py b/airflow/providers/apache/sqoop/operators/sqoop.py deleted file mode 100644 index 8aaae63dabda4..0000000000000 --- a/airflow/providers/apache/sqoop/operators/sqoop.py +++ /dev/null @@ -1,264 +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. -"""This module contains a sqoop 1 operator.""" -from __future__ import annotations - -import os -import signal -from typing import TYPE_CHECKING, Any, Sequence - -from airflow.exceptions import AirflowException -from airflow.models import BaseOperator -from airflow.providers.apache.sqoop.hooks.sqoop import SqoopHook - -if TYPE_CHECKING: - from airflow.utils.context import Context - - -class SqoopOperator(BaseOperator): - """ - Execute a Sqoop job. - - Documentation for Apache Sqoop can be found here: https://sqoop.apache.org/docs/1.4.2/SqoopUserGuide.html - - :param conn_id: str - :param cmd_type: str specify command to execute "export" or "import" - :param schema: Schema name - :param table: Table to read - :param query: Import result of arbitrary SQL query. Instead of using the table, - columns and where arguments, you can specify a SQL statement with the query - argument. Must also specify a destination directory with target_dir. - :param target_dir: HDFS destination directory where the data - from the rdbms will be written - :param append: Append data to an existing dataset in HDFS - :param file_type: "avro", "sequence", "text" Imports data to - into the specified format. Defaults to text. - :param columns: Columns to import from table - :param num_mappers: Use n mapper tasks to import/export in parallel - :param split_by: Column of the table used to split work units - :param where: WHERE clause to use during import - :param export_dir: HDFS Hive database directory to export to the rdbms - :param input_null_string: The string to be interpreted as null - for string columns - :param input_null_non_string: The string to be interpreted as null - for non-string columns - :param staging_table: The table in which data will be staged before - being inserted into the destination table - :param clear_staging_table: Indicate that any data present in the - staging table can be deleted - :param enclosed_by: Sets a required field enclosing character - :param escaped_by: Sets the escape character - :param input_fields_terminated_by: Sets the input field separator - :param input_lines_terminated_by: Sets the input end-of-line character - :param input_optionally_enclosed_by: Sets a field enclosing character - :param batch: Use batch mode for underlying statement execution - :param direct: Use direct export fast path - :param driver: Manually specify JDBC driver class to use - :param verbose: Switch to more verbose logging for debug purposes - :param relaxed_isolation: use read uncommitted isolation level - :param hcatalog_database: Specifies the database name for the HCatalog table - :param hcatalog_table: The argument value for this option is the HCatalog table - :param create_hcatalog_table: Have sqoop create the hcatalog table passed - in or not - :param properties: additional JVM properties passed to sqoop - :param extra_options: Extra import/export options to pass as dict to the SqoopHook. - If a key doesn't have a value, just pass an empty string to it. - Don't include prefix of -- for sqoop options. - :param libjars: Optional Comma separated jar files to include in the classpath. - """ - - template_fields: Sequence[str] = ( - "conn_id", - "cmd_type", - "table", - "query", - "target_dir", - "file_type", - "columns", - "split_by", - "where", - "export_dir", - "input_null_string", - "input_null_non_string", - "staging_table", - "enclosed_by", - "escaped_by", - "input_fields_terminated_by", - "input_lines_terminated_by", - "input_optionally_enclosed_by", - "properties", - "extra_options", - "driver", - "hcatalog_database", - "hcatalog_table", - "schema", - ) - template_fields_renderers = {"query": "sql"} - ui_color = "#7D8CA4" - - def __init__( - self, - *, - conn_id: str = "sqoop_default", - cmd_type: str = "import", - table: str | None = None, - query: str | None = None, - target_dir: str | None = None, - append: bool = False, - file_type: str = "text", - columns: str | None = None, - num_mappers: int | None = None, - split_by: str | None = None, - where: str | None = None, - export_dir: str | None = None, - input_null_string: str | None = None, - input_null_non_string: str | None = None, - staging_table: str | None = None, - clear_staging_table: bool = False, - enclosed_by: str | None = None, - escaped_by: str | None = None, - input_fields_terminated_by: str | None = None, - input_lines_terminated_by: str | None = None, - input_optionally_enclosed_by: str | None = None, - batch: bool = False, - direct: bool = False, - driver: Any | None = None, - verbose: bool = False, - relaxed_isolation: bool = False, - properties: dict[str, Any] | None = None, - hcatalog_database: str | None = None, - hcatalog_table: str | None = None, - create_hcatalog_table: bool = False, - extra_options: dict[str, Any] | None = None, - schema: str | None = None, - libjars: str | None = None, - **kwargs: Any, - ) -> None: - super().__init__(**kwargs) - self.conn_id = conn_id - self.cmd_type = cmd_type - self.table = table - self.query = query - self.target_dir = target_dir - self.append = append - self.file_type = file_type - self.columns = columns - self.num_mappers = num_mappers - self.split_by = split_by - self.where = where - self.export_dir = export_dir - self.input_null_string = input_null_string - self.input_null_non_string = input_null_non_string - self.staging_table = staging_table - self.clear_staging_table = clear_staging_table - self.enclosed_by = enclosed_by - self.escaped_by = escaped_by - self.input_fields_terminated_by = input_fields_terminated_by - self.input_lines_terminated_by = input_lines_terminated_by - self.input_optionally_enclosed_by = input_optionally_enclosed_by - self.batch = batch - self.direct = direct - self.driver = driver - self.verbose = verbose - self.relaxed_isolation = relaxed_isolation - self.hcatalog_database = hcatalog_database - self.hcatalog_table = hcatalog_table - self.create_hcatalog_table = create_hcatalog_table - self.properties = properties - self.extra_options = extra_options or {} - self.hook: SqoopHook | None = None - self.schema = schema - self.libjars = libjars - - def execute(self, context: Context) -> None: - """Execute sqoop job.""" - if self.hook is None: - self.hook = self._get_hook() - - if self.cmd_type == "export": - self.hook.export_table( - table=self.table, # type: ignore - export_dir=self.export_dir, - input_null_string=self.input_null_string, - input_null_non_string=self.input_null_non_string, - staging_table=self.staging_table, - clear_staging_table=self.clear_staging_table, - enclosed_by=self.enclosed_by, - escaped_by=self.escaped_by, - input_fields_terminated_by=self.input_fields_terminated_by, - input_lines_terminated_by=self.input_lines_terminated_by, - input_optionally_enclosed_by=self.input_optionally_enclosed_by, - batch=self.batch, - relaxed_isolation=self.relaxed_isolation, - schema=self.schema, - ) - elif self.cmd_type == "import": - if self.table and self.query: - raise AirflowException("Cannot specify query and table together. Need to specify either or.") - - if self.table: - self.hook.import_table( - table=self.table, - target_dir=self.target_dir, - append=self.append, - file_type=self.file_type, - columns=self.columns, - split_by=self.split_by, - where=self.where, - direct=self.direct, - driver=self.driver, - schema=self.schema, - ) - elif self.query: - self.hook.import_query( - query=self.query, - target_dir=self.target_dir, - append=self.append, - file_type=self.file_type, - split_by=self.split_by, - direct=self.direct, - driver=self.driver, - ) - else: - raise AirflowException("Provide query or table parameter to import using Sqoop") - else: - raise AirflowException("cmd_type should be 'import' or 'export'") - - def on_kill(self) -> None: - if self.hook is None: - self.hook = self._get_hook() - self.log.info("Sending SIGTERM signal to bash process group") - os.killpg(os.getpgid(self.hook.sub_process_pid), signal.SIGTERM) - - def _get_hook(self) -> SqoopHook: - """Return a SqoopHook instance.""" - # Add `create-hcatalog-table` to extra options if option passed to operator in case of `import` - # command. Similarly, if new parameters are added to the operator, you can pass them to - # `extra_options` so that you don't need to modify `SqoopHook` for each new parameter. - if self.cmd_type == "import" and self.create_hcatalog_table: - self.extra_options["create-hcatalog-table"] = "" - return SqoopHook( - conn_id=self.conn_id, - verbose=self.verbose, - num_mappers=self.num_mappers, - hcatalog_database=self.hcatalog_database, - hcatalog_table=self.hcatalog_table, - properties=self.properties, - libjars=self.libjars, - extra_options=self.extra_options, - ) diff --git a/airflow/providers/apache/sqoop/provider.yaml b/airflow/providers/apache/sqoop/provider.yaml deleted file mode 100644 index 66e12d4eb6586..0000000000000 --- a/airflow/providers/apache/sqoop/provider.yaml +++ /dev/null @@ -1,71 +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. - ---- -package-name: apache-airflow-providers-apache-sqoop -name: Apache Sqoop -description: | - `Apache Sqoop `__ - -removed: true -suspended: true -source-date-epoch: 1703288116 -versions: - - 4.2.1 - - 4.2.0 - - 4.1.0 - - 4.0.0 - - 3.2.1 - - 3.2.0 - - 3.1.1 - - 3.1.0 - - 3.0.0 - - 2.1.3 - - 2.1.2 - - 2.1.1 - - 2.1.0 - - 2.0.2 - - 2.0.1 - - 2.0.0 - - 1.0.1 - - 1.0.0 - -dependencies: - - apache-airflow>=2.6.0 - -integrations: - - integration-name: Apache Sqoop - external-doc-url: https://sqoop.apache.org/ - logo: /integration-logos/apache/sqoop.png - how-to-guide: - - /docs/apache-airflow-providers-apache-sqoop/operators.rst - tags: [apache] - -operators: - - integration-name: Apache Sqoop - python-modules: - - airflow.providers.apache.sqoop.operators.sqoop - -hooks: - - integration-name: Apache Sqoop - python-modules: - - airflow.providers.apache.sqoop.hooks.sqoop - - -connection-types: - - hook-class-name: airflow.providers.apache.sqoop.hooks.sqoop.SqoopHook - connection-type: sqoop diff --git a/airflow/providers/plexus/.latest-doc-only-change.txt b/airflow/providers/plexus/.latest-doc-only-change.txt deleted file mode 100644 index 36cd4c153e278..0000000000000 --- a/airflow/providers/plexus/.latest-doc-only-change.txt +++ /dev/null @@ -1 +0,0 @@ -99534e47f330ce0efb96402629dda5b2a4f16e8f diff --git a/airflow/providers/plexus/CHANGELOG.rst b/airflow/providers/plexus/CHANGELOG.rst deleted file mode 100644 index d576f7ef30c71..0000000000000 --- a/airflow/providers/plexus/CHANGELOG.rst +++ /dev/null @@ -1,256 +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. - - -.. 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-plexus`` - - -Changelog ---------- - -3.4.1 -..... - -.. note:: - - Mark plexus provider as removed according to `consensus on the dev@airflow.apache.org `_ - -Misc -~~~~ - -* ``Schedule plexus and apache.sqoop providers for removal (#36208)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - -3.4.0 -..... - -.. note:: - This release of provider is only available for Airflow 2.6+ as explained in the - `Apache Airflow providers support policy `_. - -Misc -~~~~ - -* ``Bump minimum Airflow version in providers to Airflow 2.6.0 (#36017)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Fix and reapply templates for provider documentation (#35686)`` - * ``Prepare docs 3rd wave of Providers October 2023 - FIX (#35233)`` - * ``Prepare docs 2nd wave of Providers November 2023 (#35836)`` - * ``Use reproducible builds for provider packages (#35693)`` - * ``Prepare docs 1st wave of Providers November 2023 (#35537)`` - * ``Prepare docs 3rd wave of Providers October 2023 (#35187)`` - * ``Pre-upgrade 'ruff==0.0.292' changes in providers (#35053)`` - -3.3.0 -..... - -.. note:: - This release of provider is only available for Airflow 2.5+ as explained in the - `Apache Airflow providers support policy `_. - -Misc -~~~~ - -* ``Bump min airflow version of providers (#34728)`` - -3.2.2 -..... - -Misc -~~~~ - -* ``Simplify conditions on len() in other providers (#33569)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Prepare docs for Aug 2023 2nd wave of Providers (#33291)`` - * ``Prepare docs for July 2023 wave of Providers (RC2) (#32381)`` - * ``D205 Support - Providers: Pagerduty to SMTP (inclusive) (#32358)`` - * ``Remove spurious headers for provider changelogs (#32373)`` - * ``Prepare docs for July 2023 wave of Providers (#32298)`` - * ``Improve provider documentation and README structure (#32125)`` - -3.2.1 -..... - -.. note:: - This release dropped support for Python 3.7 - -Misc -~~~~ - -* ``Add note about dropping Python 3.7 for providers (#32015)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Add D400 pydocstyle check - Providers (#31427)`` - -3.2.0 -..... - -.. note:: - This release of provider is only available for Airflow 2.4+ as explained in the - `Apache Airflow providers support policy `_. - -Misc -~~~~ - -* ``Bump minimum Airflow version in providers (#30917)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Add full automation for min Airflow version for providers (#30994)`` - * ``Add mechanism to suspend providers (#30422)`` - * ``Use '__version__' in providers not 'version' (#31393)`` - * ``Fixing circular import error in providers caused by airflow version check (#31379)`` - * ``Prepare docs for May 2023 wave of Providers (#31252)`` - -3.1.0 -..... - -.. note:: - This release of provider is only available for Airflow 2.3+ as explained in the - `Apache Airflow providers support policy `_. - -Misc -~~~~ - -* ``Move min airflow version to 2.3.0 for all providers (#27196)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Add documentation for July 2022 Provider's release (#25030)`` - * ``Enable string normalization in python formatting - providers (#27205)`` - * ``Update docs for September Provider's release (#26731)`` - * ``Apply PEP-563 (Postponed Evaluation of Annotations) to non-core airflow (#26289)`` - * ``Prepare docs for new providers release (August 2022) (#25618)`` - * ``Move provider dependencies to inside provider folders (#24672)`` - -3.0.0 -..... - -Breaking changes -~~~~~~~~~~~~~~~~ - -.. note:: - This release of provider is only available for Airflow 2.2+ as explained in the - `Apache Airflow providers support policy `_. - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Migrate Plexus example DAGs to new design #22457 (#24147)`` - * ``Add explanatory note for contributors about updating Changelog (#24229)`` - * ``Prepare docs for May 2022 provider's release (#24231)`` - * ``Update package description to remove double min-airflow specification (#24292)`` - -2.0.4 -..... - -Bug Fixes -~~~~~~~~~ - -* ``Fix mistakenly added install_requires for all providers (#22382)`` - -2.0.3 -..... - -Misc -~~~~~ - -* ``Add Trove classifiers in PyPI (Framework :: Apache Airflow :: Provider)`` - -2.0.2 -..... - -Misc -~~~~ - -* ``Support for Python 3.10`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Fixed changelog for January 2022 (delayed) provider's release (#21439)`` - * ``Fix K8S changelog to be PyPI-compatible (#20614)`` - * ``Add documentation for January 2021 providers release (#21257)`` - * ``Remove ':type' lines now sphinx-autoapi supports typehints (#20951)`` - * ``Update documentation for provider December 2021 release (#20523)`` - * ``Update documentation for November 2021 provider's release (#19882)`` - * ``Prepare documentation for October Provider's release (#19321)`` - * ``More f-strings (#18855)`` - * ``Update documentation for September providers release (#18613)`` - * ``Static start_date and default arg cleanup for misc. provider example DAGs (#18597)`` - -2.0.1 -..... - - -Misc -~~~~ - -* ``Optimise connection importing for Airflow 2.2.0`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Prepares docs for Rc2 release of July providers (#17116)`` - * ``Remove/refactor default_args pattern for miscellaneous providers (#16872)`` - -2.0.0 -..... - -Breaking changes -~~~~~~~~~~~~~~~~ - -* ``Auto-apply apply_default decorator (#15667)`` - -.. warning:: Due to apply_default decorator removal, this version of the provider requires Airflow 2.1.0+. - If your Airflow version is < 2.1.0, and you want to install this provider version, first upgrade - Airflow to at least version 2.1.0. Otherwise your Airflow package version will be upgraded - automatically and you will have to manually run ``airflow upgrade db`` to complete the migration. - -Features -~~~~~~~~ - -* ``Removes arrow higher limits for plexus provider (#16026)`` - -.. Below changes are excluded from the changelog. Move them to - appropriate section above if needed. Do not delete the lines(!): - * ``Adds interactivity when generating provider documentation. (#15518)`` - * ``Prepares provider release after PIP 21 compatibility (#15576)`` - * ``Remove Backport Providers (#14886)`` - * ``Update documentation for broken package releases (#14734)`` - * ``Updated documentation for June 2021 provider release (#16294)`` - * ``More documentation update for June providers release (#16405)`` - * ``Synchronizes updated changelog after buggfix release (#16464)`` - -1.0.1 -..... - -Updated documentation and readme files. - -1.0.0 -..... - -Initial version of the provider. diff --git a/airflow/providers/plexus/__init__.py b/airflow/providers/plexus/__init__.py deleted file mode 100644 index 0a4758aeaff35..0000000000000 --- a/airflow/providers/plexus/__init__.py +++ /dev/null @@ -1,42 +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. -# -# NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE -# OVERWRITTEN WHEN PREPARING DOCUMENTATION FOR THE PACKAGES. -# -# IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE -# `PROVIDER__INIT__PY_TEMPLATE.py.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY -# -from __future__ import annotations - -import packaging.version - -__all__ = ["__version__"] - -__version__ = "3.4.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( - "2.6.0" -): - raise RuntimeError( - f"The package `apache-airflow-providers-plexus:{__version__}` needs Apache Airflow 2.6.0+" - ) diff --git a/airflow/providers/plexus/hooks/__init__.py b/airflow/providers/plexus/hooks/__init__.py deleted file mode 100644 index 13a83393a9124..0000000000000 --- a/airflow/providers/plexus/hooks/__init__.py +++ /dev/null @@ -1,16 +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/plexus/hooks/plexus.py b/airflow/providers/plexus/hooks/plexus.py deleted file mode 100644 index 9432fe5fb7c8b..0000000000000 --- a/airflow/providers/plexus/hooks/plexus.py +++ /dev/null @@ -1,77 +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. -from __future__ import annotations - -from typing import Any - -import arrow -import jwt -import requests - -from airflow.exceptions import AirflowException -from airflow.hooks.base import BaseHook -from airflow.models import Variable - - -class PlexusHook(BaseHook): - """ - Used for jwt token generation and storage to make Plexus API calls. - - Requires email and password Airflow variables be created. - - Example: - - export AIRFLOW_VAR_EMAIL = user@corescientific.com - - export AIRFLOW_VAR_PASSWORD = ******* - - """ - - def __init__(self) -> None: - super().__init__() - self.__token = None - self.__token_exp = None - self.host = "https://apiplexus.corescientific.com/" - self.user_id = None - - def _generate_token(self) -> Any: - login = Variable.get("email") - pwd = Variable.get("password") - if login is None or pwd is None: - raise AirflowException("No valid email/password supplied.") - token_endpoint = self.host + "sso/jwt-token/" - response = requests.post(token_endpoint, data={"email": login, "password": pwd}, timeout=5) - if not response.ok: - raise AirflowException( - "Could not retrieve JWT Token. " - f"Status Code: [{response.status_code}]. Reason: {response.reason} - {response.text}" - ) - token = response.json()["access"] - payload = jwt.decode(token, verify=False) - self.user_id = payload["user_id"] - self.__token_exp = payload["exp"] - - return token - - @property - def token(self) -> Any: - """Returns users token.""" - if self.__token is not None: - if not self.__token_exp or arrow.get(self.__token_exp) <= arrow.now(): - self.__token = self._generate_token() - return self.__token - else: - self.__token = self._generate_token() - return self.__token diff --git a/airflow/providers/plexus/operators/__init__.py b/airflow/providers/plexus/operators/__init__.py deleted file mode 100644 index 13a83393a9124..0000000000000 --- a/airflow/providers/plexus/operators/__init__.py +++ /dev/null @@ -1,16 +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/plexus/operators/job.py b/airflow/providers/plexus/operators/job.py deleted file mode 100644 index 21002dc0766f1..0000000000000 --- a/airflow/providers/plexus/operators/job.py +++ /dev/null @@ -1,156 +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. -from __future__ import annotations - -import logging -import time -from typing import Any - -import requests - -from airflow.exceptions import AirflowException -from airflow.models import BaseOperator -from airflow.providers.plexus.hooks.plexus import PlexusHook - -logger = logging.getLogger(__name__) - - -class PlexusJobOperator(BaseOperator): - """ - Submits a Plexus job. - - :param job_params: parameters required to launch a job. - - Required job parameters are the following - - "name": job name created by user. - - "app": name of the application to run. found in Plexus UI. - - "queue": public cluster name. found in Plexus UI. - - "num_nodes": number of nodes. - - "num_cores": number of cores per node. - - """ - - def __init__(self, job_params: dict, **kwargs) -> None: - super().__init__(**kwargs) - - self.job_params = job_params - self.required_params = {"name", "app", "queue", "num_cores", "num_nodes"} - self.lookups = { - "app": ("apps/", "id", "name"), - "billing_account_id": ("users/{}/billingaccounts/", "id", None), - "queue": ("queues/", "id", "public_name"), - } - self.job_params.update({"billing_account_id": None}) - self.is_service = None - - def execute(self, context: Any) -> Any: - hook = PlexusHook() - params = self.construct_job_params(hook) - if self.is_service is True: - if self.job_params.get("expected_runtime") is None: - end_state = "Running" - else: - end_state = "Finished" - elif self.is_service is False: - end_state = "Completed" - else: - raise AirflowException( - "Unable to determine if application " - "is running as a batch job or service. " - "Contact Core Scientific AI Team." - ) - logger.info("creating job w/ following params: %s", params) - jobs_endpoint = hook.host + "jobs/" - headers = {"Authorization": f"Bearer {hook.token}"} - create_job = requests.post(jobs_endpoint, headers=headers, data=params, timeout=5) - if create_job.ok: - job = create_job.json() - jid = job["id"] - state = job["last_state"] - while state != end_state: - time.sleep(3) - jid_endpoint = jobs_endpoint + f"{jid}/" - get_job = requests.get(jid_endpoint, headers=headers, timeout=5) - if not get_job.ok: - raise AirflowException( - "Could not retrieve job status. " - f"Status Code: [{get_job.status_code}]. Reason: {get_job.reason} - {get_job.text}" - ) - new_state = get_job.json()["last_state"] - if new_state in ("Cancelled", "Failed"): - raise AirflowException(f"Job {new_state}") - elif new_state != state: - logger.info("job is %s", new_state) - state = new_state - else: - raise AirflowException( - "Could not start job. " - f"Status Code: [{create_job.status_code}]. Reason: {create_job.reason} - {create_job.text}" - ) - - def _api_lookup(self, param: str, hook): - lookup = self.lookups[param] - key = lookup[1] - mapping = None if lookup[2] is None else (lookup[2], self.job_params[param]) - - if param == "billing_account_id": - endpoint = hook.host + lookup[0].format(hook.user_id) - else: - endpoint = hook.host + lookup[0] - headers = {"Authorization": f"Bearer {hook.token}"} - response = requests.get(endpoint, headers=headers, timeout=5) - results = response.json()["results"] - - v = None - if mapping is None: - v = results[0][key] - else: - for dct in results: - if dct[mapping[0]] == mapping[1]: - v = dct[key] - if param == "app": - self.is_service = dct["is_service"] - if v is None: - raise AirflowException(f"Could not locate value for param:{key} at endpoint: {endpoint}") - - return v - - def construct_job_params(self, hook: Any) -> dict[Any, Any | None]: - """ - Creates job_params dict for api call to launch a Plexus job. - - Some parameters required to launch a job - are not available to the user in the Plexus - UI. For example, an app id is required, but - only the app name is provided in the UI. - This function acts as a backend lookup - of the required param value using the - user-provided value. - - :param hook: plexus hook object - """ - missing_params = self.required_params - set(self.job_params) - if missing_params: - raise AirflowException(f"Missing the following required job_params: {', '.join(missing_params)}") - params = {} - for prm in self.job_params: - if prm in self.lookups: - v = self._api_lookup(param=prm, hook=hook) - params[prm] = v - else: - params[prm] = self.job_params[prm] - return params diff --git a/airflow/providers/plexus/provider.yaml b/airflow/providers/plexus/provider.yaml deleted file mode 100644 index 4ae273970a460..0000000000000 --- a/airflow/providers/plexus/provider.yaml +++ /dev/null @@ -1,61 +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. - ---- -package-name: apache-airflow-providers-plexus -name: Plexus -description: | - `Plexus `__ - -removed: true -suspended: true -source-date-epoch: 1703288161 -versions: - - 3.4.1 - - 3.4.0 - - 3.3.0 - - 3.2.2 - - 3.2.1 - - 3.2.0 - - 3.1.0 - - 3.0.0 - - 2.0.4 - - 2.0.3 - - 2.0.2 - - 2.0.1 - - 2.0.0 - - 1.0.1 - - 1.0.0 - -dependencies: - - apache-airflow>=2.6.0 - - arrow>=0.16.0 - -integrations: - - integration-name: Plexus - external-doc-url: https://plexus.corescientific.com/ - logo: /integration-logos/plexus/Plexus.png - tags: [service] - -operators: - - integration-name: Plexus - python-modules: - - airflow.providers.plexus.operators.job -hooks: - - integration-name: Plexus - python-modules: - - airflow.providers.plexus.hooks.plexus diff --git a/airflow/utils/db.py b/airflow/utils/db.py index b3353f9ba0310..e2acedf6c570c 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -605,14 +605,6 @@ def create_default_connections(session: Session = NEW_SESSION): ), session, ) - merge_conn( - Connection( - conn_id="sqoop_default", - conn_type="sqoop", - host="rdbms", - ), - session, - ) merge_conn( Connection( conn_id="ssh_default", diff --git a/dev/breeze/tests/test_packages.py b/dev/breeze/tests/test_packages.py index 214b124f92dc0..eb7b3a4d8493a 100644 --- a/dev/breeze/tests/test_packages.py +++ b/dev/breeze/tests/test_packages.py @@ -109,17 +109,17 @@ def test_get_provider_requirements(): def test_get_removed_providers(): # Modify it every time we schedule provider for removal or remove it - assert ["apache.sqoop", "plexus"] == get_removed_provider_ids() + assert [] == get_removed_provider_ids() def test_get_suspended_provider_ids(): # Modify it every time we suspend/resume provider - assert ["apache.sqoop", "plexus"] == get_suspended_provider_ids() + assert [] == get_suspended_provider_ids() def test_get_suspended_provider_folders(): # Modify it every time we suspend/resume provider - assert ["apache/sqoop", "plexus"] == get_suspended_provider_folders() + assert [] == get_suspended_provider_folders() @pytest.mark.parametrize( diff --git a/docs/apache-airflow-providers-apache-sqoop/changelog.rst b/docs/apache-airflow-providers-apache-sqoop/changelog.rst deleted file mode 100644 index a11ad4bfea409..0000000000000 --- a/docs/apache-airflow-providers-apache-sqoop/changelog.rst +++ /dev/null @@ -1,25 +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. - - .. NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE - OVERWRITTEN WHEN PREPARING PACKAGES. - - .. IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE - `PROVIDER_CHANGELOG_TEMPLATE.rst.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY - -.. include:: ../../airflow/providers/apache/sqoop/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-sqoop/commits.rst b/docs/apache-airflow-providers-apache-sqoop/commits.rst deleted file mode 100644 index ce1fc492f19ad..0000000000000 --- a/docs/apache-airflow-providers-apache-sqoop/commits.rst +++ /dev/null @@ -1,343 +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. - - .. NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE - OVERWRITTEN WHEN PREPARING PACKAGES. - - .. IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE - `PROVIDER_COMMITS_TEMPLATE.rst.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY - - .. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME! - -Package apache-airflow-providers-apache-sqoop ------------------------------------------------------- - -`Apache Sqoop `__ - - -This is detailed commit list of changes for versions provider package: ``apache.sqoop``. -For high-level changelog, see :doc:`package information including changelog `. - - - -4.2.1 -..... - -Latest change: 2023-12-13 - -================================================================================================= =========== =================================================================== -Commit Committed Subject -================================================================================================= =========== =================================================================== -`97e8f58673 `_ 2023-12-13 ``Schedule plexus and apache.sqoop providers for removal (#36208)`` -================================================================================================= =========== =================================================================== - -4.2.0 -..... - -Latest change: 2023-12-08 - -================================================================================================= =========== ======================================================================= -Commit Committed Subject -================================================================================================= =========== ======================================================================= -`999b70178a `_ 2023-12-08 ``Prepare docs 1st wave of Providers December 2023 (#36112)`` -`d0918d77ee `_ 2023-12-07 ``Bump minimum Airflow version in providers to Airflow 2.6.0 (#36017)`` -`0b23d5601c `_ 2023-11-24 ``Prepare docs 2nd wave of Providers November 2023 (#35836)`` -`99534e47f3 `_ 2023-11-19 ``Use reproducible builds for provider packages (#35693)`` -`99df205f42 `_ 2023-11-16 ``Fix and reapply templates for provider documentation (#35686)`` -`1b059c57d6 `_ 2023-11-08 ``Prepare docs 1st wave of Providers November 2023 (#35537)`` -`d1c58d86de `_ 2023-10-28 ``Prepare docs 3rd wave of Providers October 2023 - FIX (#35233)`` -`3592ff4046 `_ 2023-10-28 ``Prepare docs 3rd wave of Providers October 2023 (#35187)`` -`dd7ba3cae1 `_ 2023-10-19 ``Pre-upgrade 'ruff==0.0.292' changes in providers (#35053)`` -`b75f9e8806 `_ 2023-10-18 ``Upgrade pre-commits (#35033)`` -================================================================================================= =========== ======================================================================= - -4.1.0 -..... - -Latest change: 2023-10-13 - -================================================================================================= =========== ======================================================================= -Commit Committed Subject -================================================================================================= =========== ======================================================================= -`e9987d5059 `_ 2023-10-13 ``Prepare docs 1st wave of Providers in October 2023 (#34916)`` -`0c8e30e43b `_ 2023-10-05 ``Bump min airflow version of providers (#34728)`` -`21990ed894 `_ 2023-09-08 ``Prepare docs for 09 2023 - 1st wave of Providers (#34201)`` -`c077d19060 `_ 2023-08-26 ``Prepare docs for Aug 2023 3rd wave of Providers (#33730)`` -`c645d8e40c `_ 2023-08-12 ``D401 Support - Providers: Airbyte to Atlassian (Inclusive) (#33354)`` -================================================================================================= =========== ======================================================================= - -4.0.0 -..... - -Latest change: 2023-08-05 - -================================================================================================= =========== ==================================================================================================== -Commit Committed Subject -================================================================================================= =========== ==================================================================================================== -`60677b0ba3 `_ 2023-08-05 ``Prepare docs for Aug 2023 1st wave of Providers (#33128)`` -`59f5f58e2b `_ 2023-08-03 ``Validate SqoopHook connection string and disable extra options from public hook methods (#33039)`` -`ddcd30e7c7 `_ 2023-07-31 ``Validate connection host field for Sqoop connection (#32968)`` -`225e3041d2 `_ 2023-07-06 ``Prepare docs for July 2023 wave of Providers (RC2) (#32381)`` -`3878fe6fab `_ 2023-07-05 ``Remove spurious headers for provider changelogs (#32373)`` -`cb4927a018 `_ 2023-07-05 ``Prepare docs for July 2023 wave of Providers (#32298)`` -`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` -================================================================================================= =========== ==================================================================================================== - -3.2.1 -..... - -Latest change: 2023-06-20 - -================================================================================================= =========== ============================================================== -Commit Committed Subject -================================================================================================= =========== ============================================================== -`79bcc2e668 `_ 2023-06-20 ``Prepare RC1 docs for June 2023 wave of Providers (#32001)`` -`8b146152d6 `_ 2023-06-20 ``Add note about dropping Python 3.7 for providers (#32015)`` -`13890788ae `_ 2023-06-07 ``Apache provider docstring improvements (#31730)`` -`a473facf6c `_ 2023-06-01 ``Add D400 pydocstyle check - Apache providers only (#31424)`` -================================================================================================= =========== ============================================================== - -3.2.0 -..... - -Latest change: 2023-05-19 - -================================================================================================= =========== ====================================================================================== -Commit Committed Subject -================================================================================================= =========== ====================================================================================== -`45548b9451 `_ 2023-05-19 ``Prepare RC2 docs for May 2023 wave of Providers (#31416)`` -`abea189022 `_ 2023-05-18 ``Use '__version__' in providers not 'version' (#31393)`` -`f5aed58d9f `_ 2023-05-18 ``Fixing circular import error in providers caused by airflow version check (#31379)`` -`d9ff55cf6d `_ 2023-05-16 ``Prepare docs for May 2023 wave of Providers (#31252)`` -`eef5bc7f16 `_ 2023-05-03 ``Add full automation for min Airflow version for providers (#30994)`` -`a7eb32a5b2 `_ 2023-04-30 ``Bump minimum Airflow version in providers (#30917)`` -`d23a3bbed8 `_ 2023-04-04 ``Add mechanism to suspend providers (#30422)`` -================================================================================================= =========== ====================================================================================== - -3.1.1 -..... - -Latest change: 2023-02-18 - -================================================================================================= =========== =================================================================== -Commit Committed Subject -================================================================================================= =========== =================================================================== -`470fdaea27 `_ 2023-02-18 ``Prepare docs for 02 2023 midmonth wave of Providers (#29589)`` -`655ffb835e `_ 2023-02-16 ``Move libjars parameter in Sqoop Hook to Hook parameter (#29500)`` -================================================================================================= =========== =================================================================== - -3.1.0 -..... - -Latest change: 2022-11-15 - -================================================================================================= =========== ==================================================================================== -Commit Committed Subject -================================================================================================= =========== ==================================================================================== -`12c3c39d1a `_ 2022-11-15 ``pRepare docs for November 2022 wave of Providers (#27613)`` -`78b8ea2f22 `_ 2022-10-24 ``Move min airflow version to 2.3.0 for all providers (#27196)`` -`2a34dc9e84 `_ 2022-10-23 ``Enable string normalization in python formatting - providers (#27205)`` -`f8db64c35c `_ 2022-09-28 ``Update docs for September Provider's release (#26731)`` -`06acf40a43 `_ 2022-09-13 ``Apply PEP-563 (Postponed Evaluation of Annotations) to non-core airflow (#26289)`` -`e5ac6c7cfb `_ 2022-08-10 ``Prepare docs for new providers release (August 2022) (#25618)`` -`d2459a241b `_ 2022-07-13 ``Add documentation for July 2022 Provider's release (#25030)`` -`0de31bd73a `_ 2022-06-29 ``Move provider dependencies to inside provider folders (#24672)`` -`510a6bab45 `_ 2022-06-28 ``Remove 'hook-class-names' from provider.yaml (#24702)`` -================================================================================================= =========== ==================================================================================== - -3.0.0 -..... - -Latest change: 2022-06-09 - -================================================================================================= =========== ================================================================================== -Commit Committed Subject -================================================================================================= =========== ================================================================================== -`dcdcf3a2b8 `_ 2022-06-09 ``Update release notes for RC2 release of Providers for May 2022 (#24307)`` -`717a7588bc `_ 2022-06-07 ``Update package description to remove double min-airflow specification (#24292)`` -`aeabe994b3 `_ 2022-06-07 ``Prepare docs for May 2022 provider's release (#24231)`` -`027b707d21 `_ 2022-06-05 ``Add explanatory note for contributors about updating Changelog (#24229)`` -================================================================================================= =========== ================================================================================== - -2.1.3 -..... - -Latest change: 2022-03-22 - -================================================================================================= =========== ============================================================== -Commit Committed Subject -================================================================================================= =========== ============================================================== -`d7dbfb7e26 `_ 2022-03-22 ``Add documentation for bugfix release of Providers (#22383)`` -================================================================================================= =========== ============================================================== - -2.1.2 -..... - -Latest change: 2022-03-14 - -================================================================================================= =========== ==================================================================== -Commit Committed Subject -================================================================================================= =========== ==================================================================== -`16adc035b1 `_ 2022-03-14 ``Add documentation for Classifier release for March 2022 (#22226)`` -================================================================================================= =========== ==================================================================== - -2.1.1 -..... - -Latest change: 2022-03-07 - -================================================================================================= =========== ======================================================== -Commit Committed Subject -================================================================================================= =========== ======================================================== -`f5b96315fe `_ 2022-03-07 ``Add documentation for Feb Providers release (#22056)`` -`f41bcf1e67 `_ 2022-02-09 ``add how-to guide for sqoop operator (#21424)`` -================================================================================================= =========== ======================================================== - -2.1.0 -..... - -Latest change: 2022-02-08 - -================================================================================================= =========== ========================================================================== -Commit Committed Subject -================================================================================================= =========== ========================================================================== -`d94fa37830 `_ 2022-02-08 ``Fixed changelog for January 2022 (delayed) provider's release (#21439)`` -`6c3a67d4fc `_ 2022-02-05 ``Add documentation for January 2021 providers release (#21257)`` -`39e395f981 `_ 2022-02-04 ``Add more SQL template fields renderers (#21237)`` -`602abe8394 `_ 2022-01-20 ``Remove ':type' lines now sphinx-autoapi supports typehints (#20951)`` -`f77417eb0d `_ 2021-12-31 ``Fix K8S changelog to be PyPI-compatible (#20614)`` -`97496ba2b4 `_ 2021-12-31 ``Update documentation for provider December 2021 release (#20523)`` -`d56e7b56bb `_ 2021-12-30 ``Fix template_fields type to have MyPy friendly Sequence type (#20571)`` -`485ff6cc64 `_ 2021-12-29 ``Fix MyPy errors in Apache Providers (#20422)`` -`5bb76c7780 `_ 2021-12-16 ``Fix MyPy Errors for Apache Sqoop provider. (#20304)`` -================================================================================================= =========== ========================================================================== - -2.0.2 -..... - -Latest change: 2021-08-30 - -================================================================================================= =========== ============================================================================ -Commit Committed Subject -================================================================================================= =========== ============================================================================ -`0a68588479 `_ 2021-08-30 ``Add August 2021 Provider's documentation (#17890)`` -`be75dcd39c `_ 2021-08-23 ``Update description about the new ''connection-types'' provider meta-data`` -`76ed2a49c6 `_ 2021-08-19 ``Import Hooks lazily individually in providers manager (#17682)`` -================================================================================================= =========== ============================================================================ - -2.0.1 -..... - -Latest change: 2021-07-26 - -================================================================================================= =========== =================================================================== -Commit Committed Subject -================================================================================================= =========== =================================================================== -`87f408b1e7 `_ 2021-07-26 ``Prepares docs for Rc2 release of July providers (#17116)`` -`d02ded65ea `_ 2021-07-15 ``Fixed wrongly escaped characters in amazon's changelog (#17020)`` -`b916b75079 `_ 2021-07-15 ``Prepare documentation for July release of providers. (#17015)`` -`fc917af8b4 `_ 2021-07-10 ``Fix Minor Bugs in Apache Sqoop Hook and Operator (#16350)`` -`866a601b76 `_ 2021-06-28 ``Removes pylint from our toolchain (#16682)`` -================================================================================================= =========== =================================================================== - -2.0.0 -..... - -Latest change: 2021-06-18 - -================================================================================================= =========== ======================================================================== -Commit Committed Subject -================================================================================================= =========== ======================================================================== -`bbc627a3da `_ 2021-06-18 ``Prepares documentation for rc2 release of Providers (#16501)`` -`cbf8001d76 `_ 2021-06-16 ``Synchronizes updated changelog after buggfix release (#16464)`` -`1fba5402bb `_ 2021-06-15 ``More documentation update for June providers release (#16405)`` -`9c94b72d44 `_ 2021-06-07 ``Updated documentation for June 2021 provider release (#16294)`` -`f47e10c388 `_ 2021-05-31 ``Fixes failing static checks after recent pre-commit upgrade (#16183)`` -`37681bca00 `_ 2021-05-07 ``Auto-apply apply_default decorator (#15667)`` -`807ad32ce5 `_ 2021-05-01 ``Prepares provider release after PIP 21 compatibility (#15576)`` -`40a2476a5d `_ 2021-04-28 ``Adds interactivity when generating provider documentation. (#15518)`` -`4b031d39e1 `_ 2021-04-27 ``Make Airflow code Pylint 2.8 compatible (#15534)`` -`68e4c4dcb0 `_ 2021-03-20 ``Remove Backport Providers (#14886)`` -`6e6526a0f6 `_ 2021-03-13 ``Update documentation for broken package releases (#14734)`` -================================================================================================= =========== ======================================================================== - -1.0.1 -..... - -Latest change: 2021-02-04 - -================================================================================================= =========== ======================================================== -Commit Committed Subject -================================================================================================= =========== ======================================================== -`88bdcfa0df `_ 2021-02-04 ``Prepare to release a new wave of providers. (#14013)`` -`ac2f72c98d `_ 2021-02-01 ``Implement provider versioning tools (#13767)`` -`295d66f914 `_ 2020-12-30 ``Fix Grammar in PIP warning (#13380)`` -`6cf76d7ac0 `_ 2020-12-18 ``Fix typo in pip upgrade command :( (#13148)`` -`5090fb0c89 `_ 2020-12-15 ``Add script to generate integrations.json (#13073)`` -================================================================================================= =========== ======================================================== - -1.0.0 -..... - -Latest change: 2020-12-09 - -================================================================================================= =========== ================================================================================== -Commit Committed Subject -================================================================================================= =========== ================================================================================== -`32971a1a2d `_ 2020-12-09 ``Updates providers versions to 1.0.0 (#12955)`` -`b40dffa085 `_ 2020-12-08 ``Rename remaing modules to match AIP-21 (#12917)`` -`9b39f24780 `_ 2020-12-08 ``Add support for dynamic connection form fields per provider (#12558)`` -`c34ef853c8 `_ 2020-11-20 ``Separate out documentation building per provider (#12444)`` -`0080354502 `_ 2020-11-18 ``Update provider READMEs for 1.0.0b2 batch release (#12449)`` -`ae7cb4a1e2 `_ 2020-11-17 ``Update wrong commit hash in backport provider changes (#12390)`` -`6889a333cf `_ 2020-11-15 ``Improvements for operators and hooks ref docs (#12366)`` -`7825e8f590 `_ 2020-11-13 ``Docs installation improvements (#12304)`` -`85a18e13d9 `_ 2020-11-09 ``Point at pypi project pages for cross-dependency of provider packages (#12212)`` -`59eb5de78c `_ 2020-11-09 ``Update provider READMEs for up-coming 1.0.0beta1 releases (#12206)`` -`b2a28d1590 `_ 2020-11-09 ``Moves provider packages scripts to dev (#12082)`` -`41bf172c1d `_ 2020-11-04 ``Simplify string expressions (#12093)`` -`4e8f9cc8d0 `_ 2020-11-03 ``Enable Black - Python Auto Formmatter (#9550)`` -`8c42cf1b00 `_ 2020-11-03 ``Use PyUpgrade to use Python 3.6 features (#11447)`` -`5a439e84eb `_ 2020-10-26 ``Prepare providers release 0.0.2a1 (#11855)`` -`872b1566a1 `_ 2020-10-25 ``Generated backport providers readmes/setup for 2020.10.29 (#11826)`` -`349b0811c3 `_ 2020-10-20 ``Add D200 pydocstyle check (#11688)`` -`16e7129719 `_ 2020-10-13 ``Added support for provider packages for Airflow 2.0 (#11487)`` -`0a0e1af800 `_ 2020-10-03 ``Fix Broken Markdown links in Providers README TOC (#11249)`` -`ca4238eb4d `_ 2020-10-02 ``Fixed month in backport packages to October (#11242)`` -`5220e4c384 `_ 2020-10-02 ``Prepare Backport release 2020.09.07 (#11238)`` -`fdd9b6f65b `_ 2020-08-25 ``Enable Black on Providers Packages (#10543)`` -`3696c34c28 `_ 2020-08-24 ``Fix typo in the word "release" (#10528)`` -`ee7ca128a1 `_ 2020-08-22 ``Fix broken Markdown refernces in Providers README (#10483)`` -`3b3287d7ac `_ 2020-08-05 ``Enforce keyword only arguments on apache operators (#10170)`` -`33f0cd2657 `_ 2020-07-22 ``apply_default keeps the function signature for mypy (#9784)`` -`4d74ac2111 `_ 2020-07-19 ``Increase typing for Apache and http provider package (#9729)`` -`d0e7db4024 `_ 2020-06-19 ``Fixed release number for fresh release (#9408)`` -`12af6a0800 `_ 2020-06-19 ``Final cleanup for 2020.6.23rc1 release preparation (#9404)`` -`c7e5bce57f `_ 2020-06-19 ``Prepare backport release candidate for 2020.6.23rc1 (#9370)`` -`f6bd817a3a `_ 2020-06-16 ``Introduce 'transfers' packages (#9320)`` -`0b0e4f7a4c `_ 2020-05-26 ``Preparing for RC3 relase of backports (#9026)`` -`00642a46d0 `_ 2020-05-26 ``Fixed name of 20 remaining wrongly named operators. (#8994)`` -`375d1ca229 `_ 2020-05-19 ``Release candidate 2 for backport packages 2020.05.20 (#8898)`` -`12c5e5d8ae `_ 2020-05-17 ``Prepare release candidate for backport packages (#8891)`` -`f3521fb0e3 `_ 2020-05-16 ``Regenerate readme files for backport package release (#8886)`` -`92585ca4cb `_ 2020-05-15 ``Added automated release notes generation for backport operators (#8807)`` -`4bde99f132 `_ 2020-03-23 ``Make airflow/providers pylint compatible (#7802)`` -`7e6372a681 `_ 2020-03-23 ``Add call to Super call in apache providers (#7820)`` -`97a429f9d0 `_ 2020-02-02 ``[AIRFLOW-6714] Remove magic comments about UTF-8 (#7338)`` -`0481b9a957 `_ 2020-01-12 ``[AIRFLOW-6539][AIP-21] Move Apache classes to providers.apache package (#7142)`` -================================================================================================= =========== ================================================================================== diff --git a/docs/apache-airflow-providers-apache-sqoop/index.rst b/docs/apache-airflow-providers-apache-sqoop/index.rst deleted file mode 100644 index 7a3a74514591a..0000000000000 --- a/docs/apache-airflow-providers-apache-sqoop/index.rst +++ /dev/null @@ -1,97 +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. - -``apache-airflow-providers-apache-sqoop`` -========================================= - - -.. toctree:: - :hidden: - :maxdepth: 1 - :caption: Basics - - Home - Changelog - Security - -.. toctree:: - :hidden: - :maxdepth: 1 - :caption: Guides - - Operators - - -.. toctree:: - :hidden: - :maxdepth: 1 - :caption: References - - Python API <_api/airflow/providers/apache/sqoop/index> - 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-apache-sqoop ------------------------------------------------------- - -`Apache Sqoop `__ - - -Release: 4.2.1 - -Provider package ----------------- - -This is a provider package for ``apache.sqoop`` provider. All classes for this provider package -are in ``airflow.providers.apache.sqoop`` python package. - - .. warning:: - - This provider is not maintained anymore by the community. It has been removed and is not going to be - updated anymore. The removal was done according to the process described in - `Removing community providers `_ - - Feel free to contact Airflow Development Mailing List if you have any questions. - -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-apache-sqoop`` - -Requirements ------------- - -The minimum Apache Airflow version supported by this provider package is ``2.6.0``. - -================== ================== -PIP package Version required -================== ================== -``apache-airflow`` ``>=2.6.0`` -================== ================== diff --git a/docs/apache-airflow-providers-apache-sqoop/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-sqoop/installing-providers-from-sources.rst deleted file mode 100644 index b4e730f4ff21a..0000000000000 --- a/docs/apache-airflow-providers-apache-sqoop/installing-providers-from-sources.rst +++ /dev/null @@ -1,18 +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. - -.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-sqoop/operators.rst b/docs/apache-airflow-providers-apache-sqoop/operators.rst deleted file mode 100644 index 817317526a198..0000000000000 --- a/docs/apache-airflow-providers-apache-sqoop/operators.rst +++ /dev/null @@ -1,38 +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. - -Apache Sqoop Operators -====================== - -Apache Sqoop is a tool designed to transfer data between Hadoop and relational databases. -You can use Sqoop to import data from a relational database management system (RDBMS) such as MySQL -or Oracle into the Hadoop Distributed File System (HDFS), transform the data in Hadoop MapReduce, -and then export the data back into an RDBMS. - -SqoopOperator -------------- - -Execute a Sqoop Job -^^^^^^^^^^^^^^^^^^^ - -The :class:`~airflow.providers.apache.sqoop.operators.sqoop.SqoopOperator` operator is used to -execute a sqoop job. - -Reference -^^^^^^^^^ - -For further information, look at `Sqoop User Guide `_. diff --git a/docs/apache-airflow-providers-apache-sqoop/security.rst b/docs/apache-airflow-providers-apache-sqoop/security.rst deleted file mode 100644 index afa13dac6fc9b..0000000000000 --- a/docs/apache-airflow-providers-apache-sqoop/security.rst +++ /dev/null @@ -1,18 +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. - -.. include:: ../exts/includes/security.rst diff --git a/docs/apache-airflow-providers-plexus/changelog.rst b/docs/apache-airflow-providers-plexus/changelog.rst deleted file mode 100644 index c4e0c709432e5..0000000000000 --- a/docs/apache-airflow-providers-plexus/changelog.rst +++ /dev/null @@ -1,25 +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. - - .. NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE - OVERWRITTEN WHEN PREPARING PACKAGES. - - .. IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE - `PROVIDER_CHANGELOG_TEMPLATE.rst.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY - -.. include:: ../../airflow/providers/plexus/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-plexus/commits.rst b/docs/apache-airflow-providers-plexus/commits.rst deleted file mode 100644 index ea185d00dd28a..0000000000000 --- a/docs/apache-airflow-providers-plexus/commits.rst +++ /dev/null @@ -1,282 +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. - - .. NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE - OVERWRITTEN WHEN PREPARING PACKAGES. - - .. IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE - `PROVIDER_COMMITS_TEMPLATE.rst.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY - - .. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME! - -Package apache-airflow-providers-plexus ------------------------------------------------------- - -`Plexus `__ - - -This is detailed commit list of changes for versions provider package: ``plexus``. -For high-level changelog, see :doc:`package information including changelog `. - - - -3.4.1 -..... - -Latest change: 2023-12-13 - -================================================================================================= =========== =================================================================== -Commit Committed Subject -================================================================================================= =========== =================================================================== -`97e8f58673 `_ 2023-12-13 ``Schedule plexus and apache.sqoop providers for removal (#36208)`` -================================================================================================= =========== =================================================================== - -3.4.0 -..... - -Latest change: 2023-12-08 - -================================================================================================= =========== ======================================================================= -Commit Committed Subject -================================================================================================= =========== ======================================================================= -`999b70178a `_ 2023-12-08 ``Prepare docs 1st wave of Providers December 2023 (#36112)`` -`d0918d77ee `_ 2023-12-07 ``Bump minimum Airflow version in providers to Airflow 2.6.0 (#36017)`` -`0b23d5601c `_ 2023-11-24 ``Prepare docs 2nd wave of Providers November 2023 (#35836)`` -`99534e47f3 `_ 2023-11-19 ``Use reproducible builds for provider packages (#35693)`` -`99df205f42 `_ 2023-11-16 ``Fix and reapply templates for provider documentation (#35686)`` -`1b059c57d6 `_ 2023-11-08 ``Prepare docs 1st wave of Providers November 2023 (#35537)`` -`d1c58d86de `_ 2023-10-28 ``Prepare docs 3rd wave of Providers October 2023 - FIX (#35233)`` -`3592ff4046 `_ 2023-10-28 ``Prepare docs 3rd wave of Providers October 2023 (#35187)`` -`dd7ba3cae1 `_ 2023-10-19 ``Pre-upgrade 'ruff==0.0.292' changes in providers (#35053)`` -================================================================================================= =========== ======================================================================= - -3.3.0 -..... - -Latest change: 2023-10-13 - -================================================================================================= =========== =============================================================== -Commit Committed Subject -================================================================================================= =========== =============================================================== -`e9987d5059 `_ 2023-10-13 ``Prepare docs 1st wave of Providers in October 2023 (#34916)`` -`0c8e30e43b `_ 2023-10-05 ``Bump min airflow version of providers (#34728)`` -================================================================================================= =========== =============================================================== - -3.2.2 -..... - -Latest change: 2023-08-26 - -================================================================================================= =========== ==================================================================== -Commit Committed Subject -================================================================================================= =========== ==================================================================== -`c077d19060 `_ 2023-08-26 ``Prepare docs for Aug 2023 3rd wave of Providers (#33730)`` -`1cdd82391e `_ 2023-08-21 ``Simplify conditions on len() in other providers (#33569)`` -`b5a4d36383 `_ 2023-08-11 ``Prepare docs for Aug 2023 2nd wave of Providers (#33291)`` -`225e3041d2 `_ 2023-07-06 ``Prepare docs for July 2023 wave of Providers (RC2) (#32381)`` -`e45bee8840 `_ 2023-07-05 ``D205 Support - Providers: Pagerduty to SMTP (inclusive) (#32358)`` -`3878fe6fab `_ 2023-07-05 ``Remove spurious headers for provider changelogs (#32373)`` -`cb4927a018 `_ 2023-07-05 ``Prepare docs for July 2023 wave of Providers (#32298)`` -`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` -================================================================================================= =========== ==================================================================== - -3.2.1 -..... - -Latest change: 2023-06-20 - -================================================================================================= =========== ============================================================= -Commit Committed Subject -================================================================================================= =========== ============================================================= -`79bcc2e668 `_ 2023-06-20 ``Prepare RC1 docs for June 2023 wave of Providers (#32001)`` -`8b146152d6 `_ 2023-06-20 ``Add note about dropping Python 3.7 for providers (#32015)`` -`a59076eaee `_ 2023-06-02 ``Add D400 pydocstyle check - Providers (#31427)`` -================================================================================================= =========== ============================================================= - -3.2.0 -..... - -Latest change: 2023-05-19 - -================================================================================================= =========== ====================================================================================== -Commit Committed Subject -================================================================================================= =========== ====================================================================================== -`45548b9451 `_ 2023-05-19 ``Prepare RC2 docs for May 2023 wave of Providers (#31416)`` -`abea189022 `_ 2023-05-18 ``Use '__version__' in providers not 'version' (#31393)`` -`f5aed58d9f `_ 2023-05-18 ``Fixing circular import error in providers caused by airflow version check (#31379)`` -`d9ff55cf6d `_ 2023-05-16 ``Prepare docs for May 2023 wave of Providers (#31252)`` -`eef5bc7f16 `_ 2023-05-03 ``Add full automation for min Airflow version for providers (#30994)`` -`a7eb32a5b2 `_ 2023-04-30 ``Bump minimum Airflow version in providers (#30917)`` -`d23a3bbed8 `_ 2023-04-04 ``Add mechanism to suspend providers (#30422)`` -================================================================================================= =========== ====================================================================================== - -3.1.0 -..... - -Latest change: 2022-11-15 - -================================================================================================= =========== ==================================================================================== -Commit Committed Subject -================================================================================================= =========== ==================================================================================== -`12c3c39d1a `_ 2022-11-15 ``pRepare docs for November 2022 wave of Providers (#27613)`` -`78b8ea2f22 `_ 2022-10-24 ``Move min airflow version to 2.3.0 for all providers (#27196)`` -`2a34dc9e84 `_ 2022-10-23 ``Enable string normalization in python formatting - providers (#27205)`` -`f8db64c35c `_ 2022-09-28 ``Update docs for September Provider's release (#26731)`` -`06acf40a43 `_ 2022-09-13 ``Apply PEP-563 (Postponed Evaluation of Annotations) to non-core airflow (#26289)`` -`e5ac6c7cfb `_ 2022-08-10 ``Prepare docs for new providers release (August 2022) (#25618)`` -`d2459a241b `_ 2022-07-13 ``Add documentation for July 2022 Provider's release (#25030)`` -`0de31bd73a `_ 2022-06-29 ``Move provider dependencies to inside provider folders (#24672)`` -================================================================================================= =========== ==================================================================================== - -3.0.0 -..... - -Latest change: 2022-06-09 - -================================================================================================= =========== ================================================================================== -Commit Committed Subject -================================================================================================= =========== ================================================================================== -`dcdcf3a2b8 `_ 2022-06-09 ``Update release notes for RC2 release of Providers for May 2022 (#24307)`` -`717a7588bc `_ 2022-06-07 ``Update package description to remove double min-airflow specification (#24292)`` -`aeabe994b3 `_ 2022-06-07 ``Prepare docs for May 2022 provider's release (#24231)`` -`027b707d21 `_ 2022-06-05 ``Add explanatory note for contributors about updating Changelog (#24229)`` -`88258921d8 `_ 2022-06-03 ``Migrate Plexus example DAGs to new design #22457 (#24147)`` -================================================================================================= =========== ================================================================================== - -2.0.4 -..... - -Latest change: 2022-03-22 - -================================================================================================= =========== ============================================================== -Commit Committed Subject -================================================================================================= =========== ============================================================== -`d7dbfb7e26 `_ 2022-03-22 ``Add documentation for bugfix release of Providers (#22383)`` -================================================================================================= =========== ============================================================== - -2.0.3 -..... - -Latest change: 2022-03-14 - -================================================================================================= =========== ==================================================================== -Commit Committed Subject -================================================================================================= =========== ==================================================================== -`16adc035b1 `_ 2022-03-14 ``Add documentation for Classifier release for March 2022 (#22226)`` -================================================================================================= =========== ==================================================================== - -2.0.2 -..... - -Latest change: 2022-03-07 - -================================================================================================= =========== ====================================================================================== -Commit Committed Subject -================================================================================================= =========== ====================================================================================== -`f5b96315fe `_ 2022-03-07 ``Add documentation for Feb Providers release (#22056)`` -`d94fa37830 `_ 2022-02-08 ``Fixed changelog for January 2022 (delayed) provider's release (#21439)`` -`6c3a67d4fc `_ 2022-02-05 ``Add documentation for January 2021 providers release (#21257)`` -`602abe8394 `_ 2022-01-20 ``Remove ':type' lines now sphinx-autoapi supports typehints (#20951)`` -`f77417eb0d `_ 2021-12-31 ``Fix K8S changelog to be PyPI-compatible (#20614)`` -`97496ba2b4 `_ 2021-12-31 ``Update documentation for provider December 2021 release (#20523)`` -`853576d901 `_ 2021-11-30 ``Update documentation for November 2021 provider's release (#19882)`` -`d9567eb106 `_ 2021-10-29 ``Prepare documentation for October Provider's release (#19321)`` -`86a2a19ad2 `_ 2021-10-17 ``More f-strings (#18855)`` -`840ea3efb9 `_ 2021-09-30 ``Update documentation for September providers release (#18613)`` -`ef037e7021 `_ 2021-09-29 ``Static start_date and default arg cleanup for misc. provider example DAGs (#18597)`` -================================================================================================= =========== ====================================================================================== - -2.0.1 -..... - -Latest change: 2021-08-30 - -================================================================================================= =========== ============================================================================= -Commit Committed Subject -================================================================================================= =========== ============================================================================= -`0a68588479 `_ 2021-08-30 ``Add August 2021 Provider's documentation (#17890)`` -`87f408b1e7 `_ 2021-07-26 ``Prepares docs for Rc2 release of July providers (#17116)`` -`0dbd0f420c `_ 2021-07-26 ``Remove/refactor default_args pattern for miscellaneous providers (#16872)`` -================================================================================================= =========== ============================================================================= - -2.0.0 -..... - -Latest change: 2021-06-18 - -================================================================================================= =========== ======================================================================= -Commit Committed Subject -================================================================================================= =========== ======================================================================= -`bbc627a3da `_ 2021-06-18 ``Prepares documentation for rc2 release of Providers (#16501)`` -`cbf8001d76 `_ 2021-06-16 ``Synchronizes updated changelog after buggfix release (#16464)`` -`1fba5402bb `_ 2021-06-15 ``More documentation update for June providers release (#16405)`` -`9c94b72d44 `_ 2021-06-07 ``Updated documentation for June 2021 provider release (#16294)`` -`5dd0802799 `_ 2021-05-24 ``Removes arrow higher limits for plexus provider (#16026)`` -`37681bca00 `_ 2021-05-07 ``Auto-apply apply_default decorator (#15667)`` -`807ad32ce5 `_ 2021-05-01 ``Prepares provider release after PIP 21 compatibility (#15576)`` -`40a2476a5d `_ 2021-04-28 ``Adds interactivity when generating provider documentation. (#15518)`` -`68e4c4dcb0 `_ 2021-03-20 ``Remove Backport Providers (#14886)`` -`6e6526a0f6 `_ 2021-03-13 ``Update documentation for broken package releases (#14734)`` -================================================================================================= =========== ======================================================================= - -1.0.1 -..... - -Latest change: 2021-02-04 - -================================================================================================= =========== ======================================================== -Commit Committed Subject -================================================================================================= =========== ======================================================== -`88bdcfa0df `_ 2021-02-04 ``Prepare to release a new wave of providers. (#14013)`` -`ac2f72c98d `_ 2021-02-01 ``Implement provider versioning tools (#13767)`` -`a9ac2b040b `_ 2021-01-23 ``Switch to f-strings using flynt. (#13732)`` -`3fd5ef3555 `_ 2021-01-21 ``Add missing logos for integrations (#13717)`` -`295d66f914 `_ 2020-12-30 ``Fix Grammar in PIP warning (#13380)`` -`6cf76d7ac0 `_ 2020-12-18 ``Fix typo in pip upgrade command :( (#13148)`` -================================================================================================= =========== ======================================================== - -1.0.0 -..... - -Latest change: 2020-12-09 - -================================================================================================= =========== ================================================================================== -Commit Committed Subject -================================================================================================= =========== ================================================================================== -`32971a1a2d `_ 2020-12-09 ``Updates providers versions to 1.0.0 (#12955)`` -`b40dffa085 `_ 2020-12-08 ``Rename remaing modules to match AIP-21 (#12917)`` -`c34ef853c8 `_ 2020-11-20 ``Separate out documentation building per provider (#12444)`` -`0080354502 `_ 2020-11-18 ``Update provider READMEs for 1.0.0b2 batch release (#12449)`` -`ae7cb4a1e2 `_ 2020-11-17 ``Update wrong commit hash in backport provider changes (#12390)`` -`6889a333cf `_ 2020-11-15 ``Improvements for operators and hooks ref docs (#12366)`` -`7825e8f590 `_ 2020-11-13 ``Docs installation improvements (#12304)`` -`85a18e13d9 `_ 2020-11-09 ``Point at pypi project pages for cross-dependency of provider packages (#12212)`` -`59eb5de78c `_ 2020-11-09 ``Update provider READMEs for up-coming 1.0.0beta1 releases (#12206)`` -`b2a28d1590 `_ 2020-11-09 ``Moves provider packages scripts to dev (#12082)`` -`4e8f9cc8d0 `_ 2020-11-03 ``Enable Black - Python Auto Formmatter (#9550)`` -`8c42cf1b00 `_ 2020-11-03 ``Use PyUpgrade to use Python 3.6 features (#11447)`` -`5a439e84eb `_ 2020-10-26 ``Prepare providers release 0.0.2a1 (#11855)`` -`872b1566a1 `_ 2020-10-25 ``Generated backport providers readmes/setup for 2020.10.29 (#11826)`` -`56d72e3ff8 `_ 2020-10-24 ``Replace non-empty sets with set literals (#11810)`` -`16e7129719 `_ 2020-10-13 ``Added support for provider packages for Airflow 2.0 (#11487)`` -`0a0e1af800 `_ 2020-10-03 ``Fix Broken Markdown links in Providers README TOC (#11249)`` -`ca4238eb4d `_ 2020-10-02 ``Fixed month in backport packages to October (#11242)`` -`5220e4c384 `_ 2020-10-02 ``Prepare Backport release 2020.09.07 (#11238)`` -`0161b5ea2b `_ 2020-09-26 ``Increasing type coverage for multiple provider (#11159)`` -`b9dc3c51ba `_ 2020-09-10 ``Added Plexus as an Airflow provider (#10591)`` -================================================================================================= =========== ================================================================================== diff --git a/docs/apache-airflow-providers-plexus/index.rst b/docs/apache-airflow-providers-plexus/index.rst deleted file mode 100644 index f710b1ab28d94..0000000000000 --- a/docs/apache-airflow-providers-plexus/index.rst +++ /dev/null @@ -1,105 +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. - -``apache-airflow-providers-plexus`` -=================================== - - -.. toctree:: - :hidden: - :maxdepth: 1 - :caption: Basics - - Home - Changelog - Security - -.. toctree:: - :hidden: - :maxdepth: 1 - :caption: References - - Python API <_api/airflow/providers/plexus/index> - -.. toctree:: - :hidden: - :maxdepth: 1 - :caption: System tests - - System Tests <_api/tests/system/providers/plexus/index> - -.. 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-plexus ------------------------------------------------------- - -`Plexus `__ - - -Release: 3.4.1 - -Provider package ----------------- - -This is a provider package for ``plexus`` provider. All classes for this provider package -are in ``airflow.providers.plexus`` python package. - - .. warning:: - - This provider is not maintained anymore by the community. It has been removed and is not going to be - updated anymore. The removal was done according to the process described in - `Removing community providers `_ - - Feel free to contact Airflow Development Mailing List if you have any questions. - -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-plexus`` - -Requirements ------------- - -The minimum Apache Airflow version supported by this provider package is ``2.6.0``. - -================== ================== -PIP package Version required -================== ================== -``apache-airflow`` ``>=2.6.0`` -``arrow`` ``>=0.16.0`` -================== ================== diff --git a/docs/apache-airflow-providers-plexus/installing-providers-from-sources.rst b/docs/apache-airflow-providers-plexus/installing-providers-from-sources.rst deleted file mode 100644 index b4e730f4ff21a..0000000000000 --- a/docs/apache-airflow-providers-plexus/installing-providers-from-sources.rst +++ /dev/null @@ -1,18 +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. - -.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-plexus/security.rst b/docs/apache-airflow-providers-plexus/security.rst deleted file mode 100644 index afa13dac6fc9b..0000000000000 --- a/docs/apache-airflow-providers-plexus/security.rst +++ /dev/null @@ -1,18 +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. - -.. include:: ../exts/includes/security.rst diff --git a/docs/integration-logos/apache/sqoop.png b/docs/integration-logos/apache/sqoop.png deleted file mode 100644 index b4bb262a604b5..0000000000000 Binary files a/docs/integration-logos/apache/sqoop.png and /dev/null differ diff --git a/docs/integration-logos/plexus/Plexus.png b/docs/integration-logos/plexus/Plexus.png deleted file mode 100644 index 4137a0d7f2e6f..0000000000000 Binary files a/docs/integration-logos/plexus/Plexus.png and /dev/null differ diff --git a/images/breeze/output_build-docs.svg b/images/breeze/output_build-docs.svg index f6f773ffb0953..dabcb6685415b 100644 --- a/images/breeze/output_build-docs.svg +++ b/images/breeze/output_build-docs.svg @@ -1,4 +1,4 @@ - +