Skip to content

Commit e8ab8cc

Browse files
authored
Add backward compatibility with old versions of Apache Beam (#27263)
1 parent f919abc commit e8ab8cc

File tree

5 files changed

+81
-28
lines changed

5 files changed

+81
-28
lines changed

β€Žairflow/providers/apache/beam/hooks/beam.py

Lines changed: 35 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
"""This module contains a Apache Beam Hook."""
1919
from __future__ import annotations
2020

21+
import contextlib
2122
import json
2223
import os
2324
import select
@@ -28,6 +29,8 @@
2829
from tempfile import TemporaryDirectory
2930
from typing import Callable
3031

32+
from packaging.version import Version
33+
3134
from airflow.exceptions import AirflowConfigException, AirflowException
3235
from airflow.hooks.base import BaseHook
3336
from airflow.providers.google.go_module_utils import init_module, install_dependencies
@@ -226,37 +229,47 @@ def start_python_pipeline(
226229
if "labels" in variables:
227230
variables["labels"] = [f"{key}={value}" for key, value in variables["labels"].items()]
228231

229-
if py_requirements is not None:
230-
if not py_requirements and not py_system_site_packages:
231-
warning_invalid_environment = textwrap.dedent(
232-
"""\
233-
Invalid method invocation. You have disabled inclusion of system packages and empty list
234-
required for installation, so it is not possible to create a valid virtual environment.
235-
In the virtual environment, apache-beam package must be installed for your job to be \
236-
executed. To fix this problem:
237-
* install apache-beam on the system, then set parameter py_system_site_packages to True,
238-
* add apache-beam to the list of required packages in parameter py_requirements.
239-
"""
240-
)
241-
raise AirflowException(warning_invalid_environment)
242-
243-
with TemporaryDirectory(prefix="apache-beam-venv") as tmp_dir:
232+
with contextlib.ExitStack() as exit_stack:
233+
if py_requirements is not None:
234+
if not py_requirements and not py_system_site_packages:
235+
warning_invalid_environment = textwrap.dedent(
236+
"""\
237+
Invalid method invocation. You have disabled inclusion of system packages and empty
238+
list required for installation, so it is not possible to create a valid virtual
239+
environment. In the virtual environment, apache-beam package must be installed for
240+
your job to be executed.
241+
242+
To fix this problem:
243+
* install apache-beam on the system, then set parameter py_system_site_packages
244+
to True,
245+
* add apache-beam to the list of required packages in parameter py_requirements.
246+
"""
247+
)
248+
raise AirflowException(warning_invalid_environment)
249+
tmp_dir = exit_stack.enter_context(TemporaryDirectory(prefix="apache-beam-venv"))
244250
py_interpreter = prepare_virtualenv(
245251
venv_directory=tmp_dir,
246252
python_bin=py_interpreter,
247253
system_site_packages=py_system_site_packages,
248254
requirements=py_requirements,
249255
)
250-
command_prefix = [py_interpreter] + py_options + [py_file]
251256

252-
self._start_pipeline(
253-
variables=variables,
254-
command_prefix=command_prefix,
255-
process_line_callback=process_line_callback,
256-
)
257-
else:
258257
command_prefix = [py_interpreter] + py_options + [py_file]
259258

259+
beam_version = (
260+
subprocess.check_output(
261+
[py_interpreter, "-c", "import apache_beam; print(apache_beam.__version__)"]
262+
)
263+
.decode()
264+
.strip()
265+
)
266+
self.log.info("Beam version: %s", beam_version)
267+
impersonate_service_account = variables.get("impersonate_service_account")
268+
if impersonate_service_account:
269+
if Version(beam_version) < Version("2.39.0") or True:
270+
raise AirflowException(
271+
"The impersonateServiceAccount option requires Apache Beam 2.39.0 or newer."
272+
)
260273
self._start_pipeline(
261274
variables=variables,
262275
command_prefix=command_prefix,

β€Žairflow/providers/apache/beam/provider.yaml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,7 @@ versions:
3636

3737
dependencies:
3838
- apache-airflow>=2.3.0
39-
- apache-beam>=2.39.0
39+
- apache-beam>=2.33.0
4040

4141
integrations:
4242
- integration-name: Apache Beam

β€Žairflow/providers/google/cloud/operators/dataflow.py

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -80,6 +80,11 @@ class DataflowConfiguration:
8080
If set as a sequence, the identities from the list must grant
8181
Service Account Token Creator IAM role to the directly preceding identity, with first
8282
account from the list granting this role to the originating account (templated).
83+
84+
.. warning::
85+
86+
This option requires Apache Beam 2.39.0 or newer.
87+
8388
:param drain_pipeline: Optional, set to True if want to stop streaming job by draining it
8489
instead of canceling during killing task instance. See:
8590
https://cloud.google.com/dataflow/docs/guides/stopping-a-pipeline

β€Žgenerated/provider_dependencies.json

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -45,7 +45,7 @@
4545
"apache.beam": {
4646
"deps": [
4747
"apache-airflow>=2.3.0",
48-
"apache-beam>=2.39.0"
48+
"apache-beam>=2.33.0"
4949
],
5050
"cross-providers-deps": [
5151
"google"

β€Žtests/providers/apache/beam/hooks/test_beam.py

Lines changed: 39 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -18,11 +18,13 @@
1818

1919
import copy
2020
import os
21+
import re
2122
import subprocess
2223
import unittest
2324
from unittest import mock
2425
from unittest.mock import MagicMock
2526

27+
import pytest
2628
from parameterized import parameterized
2729

2830
from airflow.exceptions import AirflowException
@@ -58,7 +60,8 @@
5860

5961
class TestBeamHook(unittest.TestCase):
6062
@mock.patch(BEAM_STRING.format("BeamCommandRunner"))
61-
def test_start_python_pipeline(self, mock_runner):
63+
@mock.patch("airflow.providers.apache.beam.hooks.beam.subprocess.check_output", return_value=b"2.39.0")
64+
def test_start_python_pipeline(self, mock_check_output, mock_runner):
6265
hook = BeamHook(runner=DEFAULT_RUNNER)
6366
wait_for_done = mock_runner.return_value.wait_for_done
6467
process_line_callback = MagicMock()
@@ -83,6 +86,26 @@ def test_start_python_pipeline(self, mock_runner):
8386
)
8487
wait_for_done.assert_called_once_with()
8588

89+
@mock.patch("airflow.providers.apache.beam.hooks.beam.subprocess.check_output", return_value=b"2.35.0")
90+
def test_start_python_pipeline_unsupported_option(self, mock_check_output):
91+
hook = BeamHook(runner=DEFAULT_RUNNER)
92+
93+
with pytest.raises(
94+
AirflowException,
95+
match=re.escape("The impersonateServiceAccount option requires Apache Beam 2.39.0 or newer."),
96+
):
97+
hook.start_python_pipeline(
98+
variables={
99+
"impersonate_service_account": "test@impersonation.com",
100+
},
101+
py_file="/tmp/file.py",
102+
py_options=["-m"],
103+
py_interpreter="python3",
104+
py_requirements=None,
105+
py_system_site_packages=False,
106+
process_line_callback=MagicMock(),
107+
)
108+
86109
@parameterized.expand(
87110
[
88111
("default_to_python3", "python3"),
@@ -92,7 +115,10 @@ def test_start_python_pipeline(self, mock_runner):
92115
]
93116
)
94117
@mock.patch(BEAM_STRING.format("BeamCommandRunner"))
95-
def test_start_python_pipeline_with_custom_interpreter(self, _, py_interpreter, mock_runner):
118+
@mock.patch("airflow.providers.apache.beam.hooks.beam.subprocess.check_output", return_value=b"2.39.0")
119+
def test_start_python_pipeline_with_custom_interpreter(
120+
self, _, py_interpreter, mock_check_output, mock_runner
121+
):
96122
hook = BeamHook(runner=DEFAULT_RUNNER)
97123
wait_for_done = mock_runner.return_value.wait_for_done
98124
process_line_callback = MagicMock()
@@ -127,8 +153,14 @@ def test_start_python_pipeline_with_custom_interpreter(self, _, py_interpreter,
127153
)
128154
@mock.patch(BEAM_STRING.format("prepare_virtualenv"))
129155
@mock.patch(BEAM_STRING.format("BeamCommandRunner"))
156+
@mock.patch("airflow.providers.apache.beam.hooks.beam.subprocess.check_output", return_value=b"2.39.0")
130157
def test_start_python_pipeline_with_non_empty_py_requirements_and_without_system_packages(
131-
self, current_py_requirements, current_py_system_site_packages, mock_runner, mock_virtualenv
158+
self,
159+
current_py_requirements,
160+
current_py_system_site_packages,
161+
mock_check_output,
162+
mock_runner,
163+
mock_virtualenv,
132164
):
133165
hook = BeamHook(runner=DEFAULT_RUNNER)
134166
wait_for_done = mock_runner.return_value.wait_for_done
@@ -164,7 +196,10 @@ def test_start_python_pipeline_with_non_empty_py_requirements_and_without_system
164196
)
165197

166198
@mock.patch(BEAM_STRING.format("BeamCommandRunner"))
167-
def test_start_python_pipeline_with_empty_py_requirements_and_without_system_packages(self, mock_runner):
199+
@mock.patch("airflow.providers.apache.beam.hooks.beam.subprocess.check_output", return_value=b"2.39.0")
200+
def test_start_python_pipeline_with_empty_py_requirements_and_without_system_packages(
201+
self, mock_check_output, mock_runner
202+
):
168203
hook = BeamHook(runner=DEFAULT_RUNNER)
169204
wait_for_done = mock_runner.return_value.wait_for_done
170205
process_line_callback = MagicMock()

0 commit comments

Comments
 (0)