Skip to content

Commit

Permalink
Move Hooks to Standard provider (#42794)
Browse files Browse the repository at this point in the history
  • Loading branch information
gopidesupavan authored Oct 10, 2024
1 parent 54005f8 commit 420b24a
Show file tree
Hide file tree
Showing 15 changed files with 94 additions and 16 deletions.
4 changes: 2 additions & 2 deletions airflow/providers_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,8 @@
from packaging.utils import canonicalize_name

from airflow.exceptions import AirflowOptionalProviderFeatureException
from airflow.hooks.filesystem import FSHook
from airflow.hooks.package_index import PackageIndexHook
from airflow.providers.standard.hooks.filesystem import FSHook
from airflow.providers.standard.hooks.package_index import PackageIndexHook
from airflow.typing_compat import ParamSpec
from airflow.utils import yaml
from airflow.utils.entry_points import entry_points_with_dist
Expand Down
2 changes: 1 addition & 1 deletion airflow/sensors/filesystem.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@

from airflow.configuration import conf
from airflow.exceptions import AirflowException
from airflow.hooks.filesystem import FSHook
from airflow.providers.standard.hooks.filesystem import FSHook
from airflow.sensors.base import BaseSensorOperator
from airflow.triggers.base import StartTriggerArgs
from airflow.triggers.file import FileTrigger
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,7 @@ Impact of Sentry on Environment variables passed to Subprocess Hook

When Sentry is enabled, by default it changes the standard library to pass all environment variables to
subprocesses opened by Airflow. This changes the default behaviour of
:class:`airflow.hooks.subprocess.SubprocessHook` - always all environment variables are passed to the
:class:`airflow.providers.standard.hooks.subprocess.SubprocessHook` - always all environment variables are passed to the
subprocess executed with specific set of environment variables. In this case not only the specified
environment variables are passed but also all existing environment variables are passed with
``SUBPROCESS_`` prefix added. This happens also for all other subprocesses.
Expand Down
4 changes: 2 additions & 2 deletions docs/apache-airflow/operators-and-hooks-ref.rst
Original file line number Diff line number Diff line change
Expand Up @@ -106,8 +106,8 @@ For details see: :doc:`apache-airflow-providers:operators-and-hooks-ref/index`.
* - Hooks
- Guides

* - :mod:`airflow.hooks.filesystem`
* - :mod:`airflow.providers.standard.hooks.filesystem`
-

* - :mod:`airflow.hooks.subprocess`
* - :mod:`airflow.providers.standard.hooks.subprocess`
-
16 changes: 16 additions & 0 deletions providers/src/airflow/providers/standard/hooks/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
File renamed without changes.
File renamed without changes.
Original file line number Diff line number Diff line change
Expand Up @@ -52,8 +52,8 @@ def run_command(
:param env: Optional dict containing environment variables to be made available to the shell
environment in which ``command`` will be executed. If omitted, ``os.environ`` will be used.
Note, that in case you have Sentry configured, original variables from the environment
will also be passed to the subprocess with ``SUBPROCESS_`` prefix. See
:doc:`/administration-and-deployment/logging-monitoring/errors` for details.
will also be passed to the subprocess with ``SUBPROCESS_`` prefix. See:
https://airflow.apache.org/docs/apache-airflow/stable/administration-and-deployment/logging-monitoring/errors.html for details.
:param output_encoding: encoding to use for decoding stdout
:param cwd: Working directory to run the command in.
If None (default), the command is run in a temporary directory.
Expand Down
2 changes: 1 addition & 1 deletion providers/src/airflow/providers/standard/operators/bash.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,8 @@
from typing import TYPE_CHECKING, Any, Callable, Container, Sequence, cast

from airflow.exceptions import AirflowException, AirflowSkipException
from airflow.hooks.subprocess import SubprocessHook
from airflow.models.baseoperator import BaseOperator
from airflow.providers.standard.hooks.subprocess import SubprocessHook
from airflow.utils.operator_helpers import context_to_airflow_vars
from airflow.utils.types import ArgNotSet

Expand Down
7 changes: 7 additions & 0 deletions providers/src/airflow/providers/standard/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -52,3 +52,10 @@ sensors:
- airflow.providers.standard.sensors.time
- airflow.providers.standard.sensors.weekday
- airflow.providers.standard.sensors.bash

hooks:
- integration-name: Standard
python-modules:
- airflow.providers.standard.hooks.filesystem
- airflow.providers.standard.hooks.package_index
- airflow.providers.standard.hooks.subprocess
16 changes: 16 additions & 0 deletions providers/tests/standard/hooks/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
39 changes: 39 additions & 0 deletions providers/tests/standard/hooks/test_filesystem.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
#
# 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 pytest

from airflow.providers.standard.hooks.filesystem import FSHook

pytestmark = pytest.mark.db_test


class TestFSHook:
def test_get_ui_field_behaviour(self):
fs_hook = FSHook()
assert fs_hook.get_ui_field_behaviour() == {
"hidden_fields": ["host", "schema", "port", "login", "password", "extra"],
"relabeling": {},
"placeholders": {},
}

def test_get_path(self):
fs_hook = FSHook(fs_conn_id="fs_default")

assert fs_hook.get_path() == "/"
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,8 @@

import pytest

from airflow.hooks.package_index import PackageIndexHook
from airflow.models.connection import Connection
from airflow.providers.standard.hooks.package_index import PackageIndexHook


class MockConnection(Connection):
Expand Down Expand Up @@ -73,7 +73,7 @@ def mock_get_connection(monkeypatch: pytest.MonkeyPatch, request: pytest.Fixture
password: str | None = testdata.get("password", None)
expected_result: str | None = testdata.get("expected_result", None)
monkeypatch.setattr(
"airflow.hooks.package_index.PackageIndexHook.get_connection",
"airflow.providers.standard.hooks.package_index.PackageIndexHook.get_connection",
lambda *_: MockConnection(host, login, password),
)
return expected_result
Expand Down Expand Up @@ -104,7 +104,7 @@ class MockProc:

return MockProc()

monkeypatch.setattr("airflow.hooks.package_index.subprocess.run", mock_run)
monkeypatch.setattr("airflow.providers.standard.hooks.package_index.subprocess.run", mock_run)

hook_instance = PackageIndexHook()
if mock_get_connection:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@

import pytest

from airflow.hooks.subprocess import SubprocessHook
from airflow.providers.standard.hooks.subprocess import SubprocessHook

OS_ENV_KEY = "SUBPROCESS_ENV_TEST"
OS_ENV_VAL = "this-is-from-os-environ"
Expand Down Expand Up @@ -81,11 +81,11 @@ def test_return_value(self, val, expected):

@mock.patch.dict("os.environ", clear=True)
@mock.patch(
"airflow.hooks.subprocess.TemporaryDirectory",
"airflow.providers.standard.hooks.subprocess.TemporaryDirectory",
return_value=MagicMock(__enter__=MagicMock(return_value="/tmp/airflowtmpcatcat")),
)
@mock.patch(
"airflow.hooks.subprocess.Popen",
"airflow.providers.standard.hooks.subprocess.Popen",
return_value=MagicMock(stdout=MagicMock(readline=MagicMock(side_effect=StopIteration), returncode=0)),
)
def test_should_exec_subprocess(self, mock_popen, mock_temporary_directory):
Expand Down
2 changes: 1 addition & 1 deletion tests/sensors/test_filesystem.py
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@
@pytest.mark.skip_if_database_isolation_mode # Test is broken in db isolation mode
class TestFileSensor:
def setup_method(self):
from airflow.hooks.filesystem import FSHook
from airflow.providers.standard.hooks.filesystem import FSHook

hook = FSHook()
args = {"owner": "airflow", "start_date": DEFAULT_DATE}
Expand Down

0 comments on commit 420b24a

Please sign in to comment.