/
beam.py
269 lines (244 loc) · 11.6 KB
/
beam.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
# 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 asyncio
from typing import Any, AsyncIterator, Sequence
from deprecated import deprecated
from google.cloud.dataflow_v1beta3 import ListJobsRequest
from airflow.exceptions import AirflowProviderDeprecationWarning
from airflow.providers.apache.beam.hooks.beam import BeamAsyncHook
from airflow.providers.google.cloud.hooks.dataflow import AsyncDataflowHook
from airflow.triggers.base import BaseTrigger, TriggerEvent
class BeamPipelineBaseTrigger(BaseTrigger):
"""Base class for Beam Pipeline Triggers."""
@staticmethod
def _get_async_hook(*args, **kwargs) -> BeamAsyncHook:
return BeamAsyncHook(*args, **kwargs)
@staticmethod
def _get_sync_dataflow_hook(**kwargs) -> AsyncDataflowHook:
return AsyncDataflowHook(**kwargs)
class BeamPythonPipelineTrigger(BeamPipelineBaseTrigger):
"""
Trigger to perform checking the Python pipeline status until it reaches terminate state.
:param variables: Variables passed to the pipeline.
:param py_file: Path to the python file to execute.
:param py_options: Additional options.
:param py_interpreter: Python version of the Apache Beam pipeline. If `None`, this defaults to the
python3. To track python versions supported by beam and related issues
check: https://issues.apache.org/jira/browse/BEAM-1251
:param py_requirements: Additional python package(s) to install.
If a value is passed to this parameter, a new virtual environment has been created with
additional packages installed.
You could also install the apache-beam package if it is not installed on your system, or you want
to use a different version.
:param py_system_site_packages: Whether to include system_site_packages in your virtualenv.
See virtualenv documentation for more information.
This option is only relevant if the ``py_requirements`` parameter is not None.
:param runner: Runner on which pipeline will be run. By default, "DirectRunner" is being used.
Other possible options: DataflowRunner, SparkRunner, FlinkRunner, PortableRunner.
See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
See: https://beam.apache.org/documentation/runners/capability-matrix/
"""
def __init__(
self,
variables: dict,
py_file: str,
py_options: list[str] | None = None,
py_interpreter: str = "python3",
py_requirements: list[str] | None = None,
py_system_site_packages: bool = False,
runner: str = "DirectRunner",
):
super().__init__()
self.variables = variables
self.py_file = py_file
self.py_options = py_options
self.py_interpreter = py_interpreter
self.py_requirements = py_requirements
self.py_system_site_packages = py_system_site_packages
self.runner = runner
def serialize(self) -> tuple[str, dict[str, Any]]:
"""Serialize BeamPythonPipelineTrigger arguments and classpath."""
return (
"airflow.providers.apache.beam.triggers.beam.BeamPythonPipelineTrigger",
{
"variables": self.variables,
"py_file": self.py_file,
"py_options": self.py_options,
"py_interpreter": self.py_interpreter,
"py_requirements": self.py_requirements,
"py_system_site_packages": self.py_system_site_packages,
"runner": self.runner,
},
)
async def run(self) -> AsyncIterator[TriggerEvent]: # type: ignore[override]
"""Get current pipeline status and yields a TriggerEvent."""
hook = self._get_async_hook(runner=self.runner)
try:
return_code = await hook.start_python_pipeline_async(
variables=self.variables,
py_file=self.py_file,
py_options=self.py_options,
py_interpreter=self.py_interpreter,
py_requirements=self.py_requirements,
py_system_site_packages=self.py_system_site_packages,
)
except Exception as e:
self.log.exception("Exception occurred while checking for pipeline state")
yield TriggerEvent({"status": "error", "message": str(e)})
else:
if return_code == 0:
yield TriggerEvent(
{
"status": "success",
"message": "Pipeline has finished SUCCESSFULLY",
}
)
else:
yield TriggerEvent({"status": "error", "message": "Operation failed"})
return
class BeamJavaPipelineTrigger(BeamPipelineBaseTrigger):
"""
Trigger to perform checking the Java pipeline status until it reaches terminate state.
:param variables: Variables passed to the job.
:param jar: Name of the jar for the pipeline.
:param job_class: Optional. Name of the java class for the pipeline.
:param runner: Runner on which pipeline will be run. By default, "DirectRunner" is being used.
Other possible options: DataflowRunner, SparkRunner, FlinkRunner, PortableRunner.
See: :class:`~providers.apache.beam.hooks.beam.BeamRunnerType`
See: https://beam.apache.org/documentation/runners/capability-matrix/
:param check_if_running: Optional. Before running job, validate that a previous run is not in process.
:param project_id: Optional. The Google Cloud project ID in which to start a job.
:param location: Optional. Job location.
:param job_name: Optional. The 'jobName' to use when executing the Dataflow job.
:param gcp_conn_id: Optional. The connection ID to use connecting to Google Cloud.
:param impersonation_chain: Optional. GCP service account to impersonate using short-term
credentials, or chained list of accounts required to get the access_token
of the last account in the list, which will be impersonated in the request.
If set as a string, the account must grant the originating account
the Service Account Token Creator IAM role.
If set as a sequence, the identities from the list must grant
Service Account Token Creator IAM role to the directly preceding identity, with first
account from the list granting this role to the originating account (templated).
:param poll_sleep: Optional. The time in seconds to sleep between polling GCP for the dataflow job status.
Default value is 10s.
:param cancel_timeout: Optional. How long (in seconds) operator should wait for the pipeline to be
successfully cancelled when task is being killed. Default value is 300s.
"""
def __init__(
self,
variables: dict,
jar: str,
job_class: str | None = None,
runner: str = "DirectRunner",
check_if_running: bool = False,
project_id: str | None = None,
location: str | None = None,
job_name: str | None = None,
gcp_conn_id: str | None = None,
impersonation_chain: str | Sequence[str] | None = None,
poll_sleep: int = 10,
cancel_timeout: int | None = None,
):
super().__init__()
self.variables = variables
self.jar = jar
self.job_class = job_class
self.runner = runner
self.check_if_running = check_if_running
self.project_id = project_id
self.location = location
self.job_name = job_name
self.gcp_conn_id = gcp_conn_id
self.impersonation_chain = impersonation_chain
self.poll_sleep = poll_sleep
self.cancel_timeout = cancel_timeout
def serialize(self) -> tuple[str, dict[str, Any]]:
"""Serialize BeamJavaPipelineTrigger arguments and classpath."""
return (
"airflow.providers.apache.beam.triggers.beam.BeamJavaPipelineTrigger",
{
"variables": self.variables,
"jar": self.jar,
"job_class": self.job_class,
"runner": self.runner,
"check_if_running": self.check_if_running,
"project_id": self.project_id,
"location": self.location,
"job_name": self.job_name,
"gcp_conn_id": self.gcp_conn_id,
"impersonation_chain": self.impersonation_chain,
"poll_sleep": self.poll_sleep,
"cancel_timeout": self.cancel_timeout,
},
)
async def run(self) -> AsyncIterator[TriggerEvent]: # type: ignore[override]
"""Get current Java pipeline status and yields a TriggerEvent."""
hook = self._get_async_hook(runner=self.runner)
return_code = 0
if self.check_if_running:
dataflow_hook = self._get_sync_dataflow_hook(
gcp_conn_id=self.gcp_conn_id,
poll_sleep=self.poll_sleep,
impersonation_chain=self.impersonation_chain,
cancel_timeout=self.cancel_timeout,
)
is_running = True
while is_running:
try:
jobs = await dataflow_hook.list_jobs(
project_id=self.project_id,
location=self.location,
jobs_filter=ListJobsRequest.Filter.ACTIVE,
)
is_running = bool([job async for job in jobs if job.name == self.job_name])
except Exception as e:
self.log.exception(f"Exception occurred while requesting jobs with name {self.job_name}")
yield TriggerEvent({"status": "error", "message": str(e)})
return
if is_running:
await asyncio.sleep(self.poll_sleep)
try:
return_code = await hook.start_java_pipeline_async(
variables=self.variables, jar=self.jar, job_class=self.job_class
)
except Exception as e:
self.log.exception("Exception occurred while starting the Java pipeline")
yield TriggerEvent({"status": "error", "message": str(e)})
if return_code == 0:
yield TriggerEvent(
{
"status": "success",
"message": "Pipeline has finished SUCCESSFULLY",
}
)
else:
yield TriggerEvent({"status": "error", "message": "Operation failed"})
return
@deprecated(
reason="`BeamPipelineTrigger` is deprecated. Please use `BeamPythonPipelineTrigger`.",
category=AirflowProviderDeprecationWarning,
)
class BeamPipelineTrigger(BeamPythonPipelineTrigger):
"""
Trigger to perform checking the Python pipeline status until it reaches terminate state.
This class is deprecated. Please use
:class:`airflow.providers.apache.beam.triggers.beam.BeamPythonPipelineTrigger`
instead.
"""
def __init__(self, *args, **kwargs):
super().__init__(*args, **kwargs)