/
dataflow.py
1175 lines (1039 loc) · 44.8 KB
/
dataflow.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
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
#
# 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 Google Dataflow Hook."""
import functools
import json
import re
import shlex
import subprocess
import time
import uuid
import warnings
from copy import deepcopy
from typing import Any, Callable, Dict, Generator, List, Optional, Sequence, Set, TypeVar, Union, cast
from googleapiclient.discovery import build
from airflow.exceptions import AirflowException
from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType, beam_options_to_args
from airflow.providers.google.common.hooks.base_google import GoogleBaseHook
from airflow.utils.log.logging_mixin import LoggingMixin
from airflow.utils.timeout import timeout
# This is the default location
# https://cloud.google.com/dataflow/pipelines/specifying-exec-params
DEFAULT_DATAFLOW_LOCATION = "us-central1"
JOB_ID_PATTERN = re.compile(
r"Submitted job: (?P<job_id_java>.*)|Created job with id: \[(?P<job_id_python>.*)\]"
)
T = TypeVar("T", bound=Callable)
def process_line_and_extract_dataflow_job_id_callback(
on_new_job_id_callback: Optional[Callable[[str], None]]
) -> Callable[[str], None]:
"""
Returns callback which triggers function passed as `on_new_job_id_callback` when Dataflow job_id is found.
To be used for `process_line_callback` in
:py:class:`~airflow.providers.apache.beam.hooks.beam.BeamCommandRunner`
:param on_new_job_id_callback: Callback called when the job ID is known
"""
def _process_line_and_extract_job_id(
line: str,
# on_new_job_id_callback: Optional[Callable[[str], None]]
) -> None:
# Job id info: https://goo.gl/SE29y9.
matched_job = JOB_ID_PATTERN.search(line)
if matched_job:
job_id = matched_job.group("job_id_java") or matched_job.group("job_id_python")
if on_new_job_id_callback:
on_new_job_id_callback(job_id)
def wrap(line: str):
return _process_line_and_extract_job_id(line)
return wrap
def _fallback_variable_parameter(parameter_name: str, variable_key_name: str) -> Callable[[T], T]:
def _wrapper(func: T) -> T:
"""
Decorator that provides fallback for location from `region` key in `variables` parameters.
:param func: function to wrap
:return: result of the function call
"""
@functools.wraps(func)
def inner_wrapper(self: "DataflowHook", *args, **kwargs):
if args:
raise AirflowException(
"You must use keyword arguments in this methods rather than positional"
)
parameter_location = kwargs.get(parameter_name)
variables_location = kwargs.get("variables", {}).get(variable_key_name)
if parameter_location and variables_location:
raise AirflowException(
f"The mutually exclusive parameter `{parameter_name}` and `{variable_key_name}` key "
f"in `variables` parameter are both present. Please remove one."
)
if parameter_location or variables_location:
kwargs[parameter_name] = parameter_location or variables_location
if variables_location:
copy_variables = deepcopy(kwargs["variables"])
del copy_variables[variable_key_name]
kwargs["variables"] = copy_variables
return func(self, *args, **kwargs)
return cast(T, inner_wrapper)
return _wrapper
_fallback_to_location_from_variables = _fallback_variable_parameter("location", "region")
_fallback_to_project_id_from_variables = _fallback_variable_parameter("project_id", "project")
class DataflowJobStatus:
"""
Helper class with Dataflow job statuses.
Reference: https://cloud.google.com/dataflow/docs/reference/rest/v1b3/projects.jobs#Job.JobState
"""
JOB_STATE_DONE = "JOB_STATE_DONE"
JOB_STATE_UNKNOWN = "JOB_STATE_UNKNOWN"
JOB_STATE_STOPPED = "JOB_STATE_STOPPED"
JOB_STATE_RUNNING = "JOB_STATE_RUNNING"
JOB_STATE_FAILED = "JOB_STATE_FAILED"
JOB_STATE_CANCELLED = "JOB_STATE_CANCELLED"
JOB_STATE_UPDATED = "JOB_STATE_UPDATED"
JOB_STATE_DRAINING = "JOB_STATE_DRAINING"
JOB_STATE_DRAINED = "JOB_STATE_DRAINED"
JOB_STATE_PENDING = "JOB_STATE_PENDING"
JOB_STATE_CANCELLING = "JOB_STATE_CANCELLING"
JOB_STATE_QUEUED = "JOB_STATE_QUEUED"
FAILED_END_STATES = {JOB_STATE_FAILED, JOB_STATE_CANCELLED}
SUCCEEDED_END_STATES = {JOB_STATE_DONE, JOB_STATE_UPDATED, JOB_STATE_DRAINED}
TERMINAL_STATES = SUCCEEDED_END_STATES | FAILED_END_STATES
AWAITING_STATES = {
JOB_STATE_RUNNING,
JOB_STATE_PENDING,
JOB_STATE_QUEUED,
JOB_STATE_CANCELLING,
JOB_STATE_DRAINING,
JOB_STATE_STOPPED,
}
class DataflowJobType:
"""Helper class with Dataflow job types."""
JOB_TYPE_UNKNOWN = "JOB_TYPE_UNKNOWN"
JOB_TYPE_BATCH = "JOB_TYPE_BATCH"
JOB_TYPE_STREAMING = "JOB_TYPE_STREAMING"
class _DataflowJobsController(LoggingMixin):
"""
Interface for communication with Google API.
It's not use Apache Beam, but only Google Dataflow API.
:param dataflow: Discovery resource
:param project_number: The Google Cloud Project ID.
:param location: Job location.
:param poll_sleep: The status refresh rate for pending operations.
:param name: The Job ID prefix used when the multiple_jobs option is passed is set to True.
:param job_id: ID of a single job.
:param num_retries: Maximum number of retries in case of connection problems.
:param multiple_jobs: If set to true this task will be searched by name prefix (``name`` parameter),
not by specific job ID, then actions will be performed on all matching jobs.
:param drain_pipeline: Optional, set to True if want to stop streaming job by draining it
instead of canceling.
:param cancel_timeout: wait time in seconds for successful job canceling
:param wait_until_finished: If True, wait for the end of pipeline execution before exiting. If False,
it only submits job and check once is job not in terminal state.
The default behavior depends on the type of pipeline:
* for the streaming pipeline, wait for jobs to start,
* for the batch pipeline, wait for the jobs to complete.
"""
def __init__(
self,
dataflow: Any,
project_number: str,
location: str,
poll_sleep: int = 10,
name: Optional[str] = None,
job_id: Optional[str] = None,
num_retries: int = 0,
multiple_jobs: bool = False,
drain_pipeline: bool = False,
cancel_timeout: Optional[int] = 5 * 60,
wait_until_finished: Optional[bool] = None,
) -> None:
super().__init__()
self._dataflow = dataflow
self._project_number = project_number
self._job_name = name
self._job_location = location
self._multiple_jobs = multiple_jobs
self._job_id = job_id
self._num_retries = num_retries
self._poll_sleep = poll_sleep
self._cancel_timeout = cancel_timeout
self._jobs: Optional[List[dict]] = None
self.drain_pipeline = drain_pipeline
self._wait_until_finished = wait_until_finished
def is_job_running(self) -> bool:
"""
Helper method to check if jos is still running in dataflow
:return: True if job is running.
:rtype: bool
"""
self._refresh_jobs()
if not self._jobs:
return False
for job in self._jobs:
if job["currentState"] not in DataflowJobStatus.TERMINAL_STATES:
return True
return False
def _get_current_jobs(self) -> List[dict]:
"""
Helper method to get list of jobs that start with job name or id
:return: list of jobs including id's
:rtype: list
"""
if not self._multiple_jobs and self._job_id:
return [self.fetch_job_by_id(self._job_id)]
elif self._job_name:
jobs = self._fetch_jobs_by_prefix_name(self._job_name.lower())
if len(jobs) == 1:
self._job_id = jobs[0]["id"]
return jobs
else:
raise Exception("Missing both dataflow job ID and name.")
def fetch_job_by_id(self, job_id: str) -> dict:
"""
Helper method to fetch the job with the specified Job ID.
:param job_id: Job ID to get.
:return: the Job
:rtype: dict
"""
return (
self._dataflow.projects()
.locations()
.jobs()
.get(
projectId=self._project_number,
location=self._job_location,
jobId=job_id,
)
.execute(num_retries=self._num_retries)
)
def fetch_job_metrics_by_id(self, job_id: str) -> dict:
"""
Helper method to fetch the job metrics with the specified Job ID.
:param job_id: Job ID to get.
:return: the JobMetrics. See:
https://cloud.google.com/dataflow/docs/reference/rest/v1b3/JobMetrics
:rtype: dict
"""
result = (
self._dataflow.projects()
.locations()
.jobs()
.getMetrics(projectId=self._project_number, location=self._job_location, jobId=job_id)
.execute(num_retries=self._num_retries)
)
self.log.debug("fetch_job_metrics_by_id %s:\n%s", job_id, result)
return result
def _fetch_list_job_messages_responses(self, job_id: str) -> Generator[dict, None, None]:
"""
Helper method to fetch ListJobMessagesResponse with the specified Job ID.
:param job_id: Job ID to get.
:return: yields the ListJobMessagesResponse. See:
https://cloud.google.com/dataflow/docs/reference/rest/v1b3/ListJobMessagesResponse
:rtype: Generator[dict, None, None]
"""
request = (
self._dataflow.projects()
.locations()
.jobs()
.messages()
.list(projectId=self._project_number, location=self._job_location, jobId=job_id)
)
while request is not None:
response = request.execute(num_retries=self._num_retries)
yield response
request = (
self._dataflow.projects()
.locations()
.jobs()
.messages()
.list_next(previous_request=request, previous_response=response)
)
def fetch_job_messages_by_id(self, job_id: str) -> List[dict]:
"""
Helper method to fetch the job messages with the specified Job ID.
:param job_id: Job ID to get.
:return: the list of JobMessages. See:
https://cloud.google.com/dataflow/docs/reference/rest/v1b3/ListJobMessagesResponse#JobMessage
:rtype: List[dict]
"""
messages: List[dict] = []
for response in self._fetch_list_job_messages_responses(job_id=job_id):
messages.extend(response.get("jobMessages", []))
return messages
def fetch_job_autoscaling_events_by_id(self, job_id: str) -> List[dict]:
"""
Helper method to fetch the job autoscaling events with the specified Job ID.
:param job_id: Job ID to get.
:return: the list of AutoscalingEvents. See:
https://cloud.google.com/dataflow/docs/reference/rest/v1b3/ListJobMessagesResponse#autoscalingevent
:rtype: List[dict]
"""
autoscaling_events: List[dict] = []
for response in self._fetch_list_job_messages_responses(job_id=job_id):
autoscaling_events.extend(response.get("autoscalingEvents", []))
return autoscaling_events
def _fetch_all_jobs(self) -> List[dict]:
request = (
self._dataflow.projects()
.locations()
.jobs()
.list(projectId=self._project_number, location=self._job_location)
)
all_jobs: List[dict] = []
while request is not None:
response = request.execute(num_retries=self._num_retries)
jobs = response.get("jobs")
if jobs is None:
break
all_jobs.extend(jobs)
request = (
self._dataflow.projects()
.locations()
.jobs()
.list_next(previous_request=request, previous_response=response)
)
return all_jobs
def _fetch_jobs_by_prefix_name(self, prefix_name: str) -> List[dict]:
jobs = self._fetch_all_jobs()
jobs = [job for job in jobs if job["name"].startswith(prefix_name)]
return jobs
def _refresh_jobs(self) -> None:
"""
Helper method to get all jobs by name
:return: jobs
:rtype: list
"""
self._jobs = self._get_current_jobs()
if self._jobs:
for job in self._jobs:
self.log.info(
"Google Cloud DataFlow job %s is state: %s",
job["name"],
job["currentState"],
)
else:
self.log.info("Google Cloud DataFlow job not available yet..")
def _check_dataflow_job_state(self, job) -> bool:
"""
Helper method to check the state of one job in dataflow for this task
if job failed raise exception
:return: True if job is done.
:rtype: bool
:raise: Exception
"""
if self._wait_until_finished is None:
wait_for_running = job.get('type') == DataflowJobType.JOB_TYPE_STREAMING
else:
wait_for_running = not self._wait_until_finished
if job['currentState'] == DataflowJobStatus.JOB_STATE_DONE:
return True
elif job['currentState'] == DataflowJobStatus.JOB_STATE_FAILED:
raise Exception(f"Google Cloud Dataflow job {job['name']} has failed.")
elif job['currentState'] == DataflowJobStatus.JOB_STATE_CANCELLED:
raise Exception(f"Google Cloud Dataflow job {job['name']} was cancelled.")
elif job['currentState'] == DataflowJobStatus.JOB_STATE_DRAINED:
raise Exception(f"Google Cloud Dataflow job {job['name']} was drained.")
elif job['currentState'] == DataflowJobStatus.JOB_STATE_UPDATED:
raise Exception(f"Google Cloud Dataflow job {job['name']} was updated.")
elif job['currentState'] == DataflowJobStatus.JOB_STATE_RUNNING and wait_for_running:
return True
elif job['currentState'] in DataflowJobStatus.AWAITING_STATES:
return self._wait_until_finished is False
self.log.debug("Current job: %s", str(job))
raise Exception(f"Google Cloud Dataflow job {job['name']} was unknown state: {job['currentState']}")
def wait_for_done(self) -> None:
"""Helper method to wait for result of submitted job."""
self.log.info("Start waiting for done.")
self._refresh_jobs()
while self._jobs and not all(self._check_dataflow_job_state(job) for job in self._jobs):
self.log.info("Waiting for done. Sleep %s s", self._poll_sleep)
time.sleep(self._poll_sleep)
self._refresh_jobs()
def get_jobs(self, refresh: bool = False) -> List[dict]:
"""
Returns Dataflow jobs.
:param refresh: Forces the latest data to be fetched.
:return: list of jobs
:rtype: list
"""
if not self._jobs or refresh:
self._refresh_jobs()
if not self._jobs:
raise ValueError("Could not read _jobs")
return self._jobs
def _wait_for_states(self, expected_states: Set[str]):
"""Waiting for the jobs to reach a certain state."""
if not self._jobs:
raise ValueError("The _jobs should be set")
while True:
self._refresh_jobs()
job_states = {job['currentState'] for job in self._jobs}
if not job_states.difference(expected_states):
return
unexpected_failed_end_states = expected_states - DataflowJobStatus.FAILED_END_STATES
if unexpected_failed_end_states.intersection(job_states):
unexpected_failed_jobs = {
job for job in self._jobs if job['currentState'] in unexpected_failed_end_states
}
raise AirflowException(
"Jobs failed: "
+ ", ".join(
f"ID: {job['id']} name: {job['name']} state: {job['currentState']}"
for job in unexpected_failed_jobs
)
)
time.sleep(self._poll_sleep)
def cancel(self) -> None:
"""Cancels or drains current job"""
jobs = self.get_jobs()
job_ids = [job["id"] for job in jobs if job["currentState"] not in DataflowJobStatus.TERMINAL_STATES]
if job_ids:
batch = self._dataflow.new_batch_http_request()
self.log.info("Canceling jobs: %s", ", ".join(job_ids))
for job in jobs:
requested_state = (
DataflowJobStatus.JOB_STATE_DRAINED
if self.drain_pipeline and job["type"] == DataflowJobType.JOB_TYPE_STREAMING
else DataflowJobStatus.JOB_STATE_CANCELLED
)
batch.add(
self._dataflow.projects()
.locations()
.jobs()
.update(
projectId=self._project_number,
location=self._job_location,
jobId=job["id"],
body={"requestedState": requested_state},
)
)
batch.execute()
if self._cancel_timeout and isinstance(self._cancel_timeout, int):
timeout_error_message = (
f"Canceling jobs failed due to timeout ({self._cancel_timeout}s): {', '.join(job_ids)}"
)
tm = timeout(seconds=self._cancel_timeout, error_message=timeout_error_message)
with tm:
self._wait_for_states({DataflowJobStatus.JOB_STATE_CANCELLED})
else:
self.log.info("No jobs to cancel")
class DataflowHook(GoogleBaseHook):
"""
Hook for Google Dataflow.
All the methods in the hook where project_id is used must be called with
keyword arguments rather than positional.
"""
def __init__(
self,
gcp_conn_id: str = "google_cloud_default",
delegate_to: Optional[str] = None,
poll_sleep: int = 10,
impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
drain_pipeline: bool = False,
cancel_timeout: Optional[int] = 5 * 60,
wait_until_finished: Optional[bool] = None,
) -> None:
self.poll_sleep = poll_sleep
self.drain_pipeline = drain_pipeline
self.cancel_timeout = cancel_timeout
self.wait_until_finished = wait_until_finished
self.job_id: Optional[str] = None
self.beam_hook = BeamHook(BeamRunnerType.DataflowRunner)
super().__init__(
gcp_conn_id=gcp_conn_id,
delegate_to=delegate_to,
impersonation_chain=impersonation_chain,
)
def get_conn(self) -> build:
"""Returns a Google Cloud Dataflow service object."""
http_authorized = self._authorize()
return build("dataflow", "v1b3", http=http_authorized, cache_discovery=False)
@_fallback_to_location_from_variables
@_fallback_to_project_id_from_variables
@GoogleBaseHook.fallback_to_default_project_id
def start_java_dataflow(
self,
job_name: str,
variables: dict,
jar: str,
project_id: str,
job_class: Optional[str] = None,
append_job_name: bool = True,
multiple_jobs: bool = False,
on_new_job_id_callback: Optional[Callable[[str], None]] = None,
location: str = DEFAULT_DATAFLOW_LOCATION,
) -> None:
"""
Starts Dataflow java job.
:param job_name: The name of the job.
:param variables: Variables passed to the job.
:param project_id: Optional, the Google Cloud project ID in which to start a job.
If set to None or missing, the default project_id from the Google Cloud connection is used.
:param jar: Name of the jar for the job
:param job_class: Name of the java class for the job.
:param append_job_name: True if unique suffix has to be appended to job name.
:param multiple_jobs: True if to check for multiple job in dataflow
:param on_new_job_id_callback: Callback called when the job ID is known.
:param location: Job location.
"""
warnings.warn(
""""This method is deprecated.
Please use `airflow.providers.apache.beam.hooks.beam.start.start_java_pipeline`
to start pipeline and `providers.google.cloud.hooks.dataflow.DataflowHook.wait_for_done`
to wait for the required pipeline state.
""",
DeprecationWarning,
stacklevel=3,
)
name = self.build_dataflow_job_name(job_name, append_job_name)
variables["jobName"] = name
variables["region"] = location
variables["project"] = project_id
if "labels" in variables:
variables["labels"] = json.dumps(variables["labels"], separators=(",", ":"))
self.beam_hook.start_java_pipeline(
variables=variables,
jar=jar,
job_class=job_class,
process_line_callback=process_line_and_extract_dataflow_job_id_callback(on_new_job_id_callback),
)
self.wait_for_done(
job_name=name,
location=location,
job_id=self.job_id,
multiple_jobs=multiple_jobs,
)
@_fallback_to_location_from_variables
@_fallback_to_project_id_from_variables
@GoogleBaseHook.fallback_to_default_project_id
def start_template_dataflow(
self,
job_name: str,
variables: dict,
parameters: dict,
dataflow_template: str,
project_id: str,
append_job_name: bool = True,
on_new_job_id_callback: Optional[Callable[[str], None]] = None,
on_new_job_callback: Optional[Callable[[dict], None]] = None,
location: str = DEFAULT_DATAFLOW_LOCATION,
environment: Optional[dict] = None,
) -> dict:
"""
Starts Dataflow template job.
:param job_name: The name of the job.
:param variables: Map of job runtime environment options.
It will update environment argument if passed.
.. seealso::
For more information on possible configurations, look at the API documentation
`https://cloud.google.com/dataflow/pipelines/specifying-exec-params
<https://cloud.google.com/dataflow/docs/reference/rest/v1b3/RuntimeEnvironment>`__
:param parameters: Parameters for the template
:param dataflow_template: GCS path to the template.
:param project_id: Optional, the Google Cloud project ID in which to start a job.
If set to None or missing, the default project_id from the Google Cloud connection is used.
:param append_job_name: True if unique suffix has to be appended to job name.
:param on_new_job_id_callback: (Deprecated) Callback called when the Job is known.
:param on_new_job_callback: Callback called when the Job is known.
:param location: Job location.
.. seealso::
For more information on possible configurations, look at the API documentation
`https://cloud.google.com/dataflow/pipelines/specifying-exec-params
<https://cloud.google.com/dataflow/docs/reference/rest/v1b3/RuntimeEnvironment>`__
"""
name = self.build_dataflow_job_name(job_name, append_job_name)
environment = environment or {}
# available keys for runtime environment are listed here:
# https://cloud.google.com/dataflow/docs/reference/rest/v1b3/RuntimeEnvironment
environment_keys = [
"numWorkers",
"maxWorkers",
"zone",
"serviceAccountEmail",
"tempLocation",
"bypassTempDirValidation",
"machineType",
"additionalExperiments",
"network",
"subnetwork",
"additionalUserLabels",
"kmsKeyName",
"ipConfiguration",
"workerRegion",
"workerZone",
]
for key in variables:
if key in environment_keys:
if key in environment:
self.log.warning(
"'%s' parameter in 'variables' will override of "
"the same one passed in 'environment'!",
key,
)
environment.update({key: variables[key]})
service = self.get_conn()
request = (
service.projects()
.locations()
.templates()
.launch(
projectId=project_id,
location=location,
gcsPath=dataflow_template,
body={
"jobName": name,
"parameters": parameters,
"environment": environment,
},
)
)
response = request.execute(num_retries=self.num_retries)
job = response["job"]
if on_new_job_id_callback:
warnings.warn(
"on_new_job_id_callback is Deprecated. Please start using on_new_job_callback",
DeprecationWarning,
stacklevel=3,
)
on_new_job_id_callback(job.get("id"))
if on_new_job_callback:
on_new_job_callback(job)
jobs_controller = _DataflowJobsController(
dataflow=self.get_conn(),
project_number=project_id,
name=name,
job_id=job["id"],
location=location,
poll_sleep=self.poll_sleep,
num_retries=self.num_retries,
drain_pipeline=self.drain_pipeline,
cancel_timeout=self.cancel_timeout,
wait_until_finished=self.wait_until_finished,
)
jobs_controller.wait_for_done()
return response["job"]
@GoogleBaseHook.fallback_to_default_project_id
def start_flex_template(
self,
body: dict,
location: str,
project_id: str,
on_new_job_id_callback: Optional[Callable[[str], None]] = None,
on_new_job_callback: Optional[Callable[[dict], None]] = None,
):
"""
Starts flex templates with the Dataflow pipeline.
:param body: The request body. See:
https://cloud.google.com/dataflow/docs/reference/rest/v1b3/projects.locations.flexTemplates/launch#request-body
:param location: The location of the Dataflow job (for example europe-west1)
:param project_id: The ID of the GCP project that owns the job.
If set to ``None`` or missing, the default project_id from the GCP connection is used.
:param on_new_job_id_callback: (Deprecated) A callback that is called when a Job ID is detected.
:param on_new_job_callback: A callback that is called when a Job is detected.
:return: the Job
"""
service = self.get_conn()
request = (
service.projects()
.locations()
.flexTemplates()
.launch(projectId=project_id, body=body, location=location)
)
response = request.execute(num_retries=self.num_retries)
job = response["job"]
if on_new_job_id_callback:
warnings.warn(
"on_new_job_id_callback is Deprecated. Please start using on_new_job_callback",
DeprecationWarning,
stacklevel=3,
)
on_new_job_id_callback(job.get("id"))
if on_new_job_callback:
on_new_job_callback(job)
jobs_controller = _DataflowJobsController(
dataflow=self.get_conn(),
project_number=project_id,
job_id=job.get("id"),
location=location,
poll_sleep=self.poll_sleep,
num_retries=self.num_retries,
cancel_timeout=self.cancel_timeout,
wait_until_finished=self.wait_until_finished,
)
jobs_controller.wait_for_done()
return jobs_controller.get_jobs(refresh=True)[0]
@_fallback_to_location_from_variables
@_fallback_to_project_id_from_variables
@GoogleBaseHook.fallback_to_default_project_id
def start_python_dataflow(
self,
job_name: str,
variables: dict,
dataflow: str,
py_options: List[str],
project_id: str,
py_interpreter: str = "python3",
py_requirements: Optional[List[str]] = None,
py_system_site_packages: bool = False,
append_job_name: bool = True,
on_new_job_id_callback: Optional[Callable[[str], None]] = None,
location: str = DEFAULT_DATAFLOW_LOCATION,
):
"""
Starts Dataflow job.
:param job_name: The name of the job.
:param variables: Variables passed to the job.
:param dataflow: Name of the Dataflow process.
:param py_options: Additional options.
:param project_id: The ID of the GCP project that owns the job.
If set to ``None`` or missing, the default project_id from the GCP connection is used.
:param py_interpreter: Python version of the 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 append_job_name: True if unique suffix has to be appended to job name.
:param project_id: Optional, the Google Cloud project ID in which to start a job.
If set to None or missing, the default project_id from the Google Cloud connection is used.
:param on_new_job_id_callback: Callback called when the job ID is known.
:param location: Job location.
"""
warnings.warn(
"""This method is deprecated.
Please use `airflow.providers.apache.beam.hooks.beam.start.start_python_pipeline`
to start pipeline and `providers.google.cloud.hooks.dataflow.DataflowHook.wait_for_done`
to wait for the required pipeline state.
""",
DeprecationWarning,
stacklevel=3,
)
name = self.build_dataflow_job_name(job_name, append_job_name)
variables["job_name"] = name
variables["region"] = location
variables["project"] = project_id
self.beam_hook.start_python_pipeline(
variables=variables,
py_file=dataflow,
py_options=py_options,
py_interpreter=py_interpreter,
py_requirements=py_requirements,
py_system_site_packages=py_system_site_packages,
process_line_callback=process_line_and_extract_dataflow_job_id_callback(on_new_job_id_callback),
)
self.wait_for_done(
job_name=name,
location=location,
job_id=self.job_id,
)
@staticmethod
def build_dataflow_job_name(job_name: str, append_job_name: bool = True) -> str:
"""Builds Dataflow job name."""
base_job_name = str(job_name).replace("_", "-")
if not re.match(r"^[a-z]([-a-z0-9]*[a-z0-9])?$", base_job_name):
raise ValueError(
f"Invalid job_name ({base_job_name}); the name must consist of only the characters "
f"[-a-z0-9], starting with a letter and ending with a letter or number "
)
if append_job_name:
safe_job_name = base_job_name + "-" + str(uuid.uuid4())[:8]
else:
safe_job_name = base_job_name
return safe_job_name
@_fallback_to_location_from_variables
@_fallback_to_project_id_from_variables
@GoogleBaseHook.fallback_to_default_project_id
def is_job_dataflow_running(
self,
name: str,
project_id: str,
location: str = DEFAULT_DATAFLOW_LOCATION,
variables: Optional[dict] = None,
) -> bool:
"""
Helper method to check if jos is still running in dataflow
:param name: The name of the job.
:param project_id: Optional, the Google Cloud project ID in which to start a job.
If set to None or missing, the default project_id from the Google Cloud connection is used.
:param location: Job location.
:return: True if job is running.
:rtype: bool
"""
if variables:
warnings.warn(
"The variables parameter has been deprecated. You should pass location using "
"the location parameter.",
DeprecationWarning,
stacklevel=4,
)
jobs_controller = _DataflowJobsController(
dataflow=self.get_conn(),
project_number=project_id,
name=name,
location=location,
poll_sleep=self.poll_sleep,
drain_pipeline=self.drain_pipeline,
num_retries=self.num_retries,
cancel_timeout=self.cancel_timeout,
)
return jobs_controller.is_job_running()
@GoogleBaseHook.fallback_to_default_project_id
def cancel_job(
self,
project_id: str,
job_name: Optional[str] = None,
job_id: Optional[str] = None,
location: str = DEFAULT_DATAFLOW_LOCATION,
) -> None:
"""
Cancels the job with the specified name prefix or Job ID.
Parameter ``name`` and ``job_id`` are mutually exclusive.
:param job_name: Name prefix specifying which jobs are to be canceled.
:param job_id: Job ID specifying which jobs are to be canceled.
:param location: Job location.
:param project_id: Optional, the Google Cloud project ID in which to start a job.
If set to None or missing, the default project_id from the Google Cloud connection is used.
"""
jobs_controller = _DataflowJobsController(
dataflow=self.get_conn(),
project_number=project_id,
name=job_name,
job_id=job_id,
location=location,
poll_sleep=self.poll_sleep,
drain_pipeline=self.drain_pipeline,
num_retries=self.num_retries,
cancel_timeout=self.cancel_timeout,
)
jobs_controller.cancel()
@GoogleBaseHook.fallback_to_default_project_id
def start_sql_job(
self,
job_name: str,
query: str,
options: Dict[str, Any],
project_id: str,
location: str = DEFAULT_DATAFLOW_LOCATION,
on_new_job_id_callback: Optional[Callable[[str], None]] = None,
on_new_job_callback: Optional[Callable[[dict], None]] = None,
):
"""
Starts Dataflow SQL query.
:param job_name: The unique name to assign to the Cloud Dataflow job.
:param query: The SQL query to execute.
:param options: Job parameters to be executed.
For more information, look at:
`https://cloud.google.com/sdk/gcloud/reference/beta/dataflow/sql/query
<gcloud beta dataflow sql query>`__
command reference
:param location: The location of the Dataflow job (for example europe-west1)
:param project_id: The ID of the GCP project that owns the job.
If set to ``None`` or missing, the default project_id from the GCP connection is used.
:param on_new_job_id_callback: (Deprecated) Callback called when the job ID is known.
:param on_new_job_callback: Callback called when the job is known.
:return: the new job object
"""
gcp_options = [
f"--project={project_id}",
"--format=value(job.id)",
f"--job-name={job_name}",
f"--region={location}",
]
if self.impersonation_chain:
if isinstance(self.impersonation_chain, str):
impersonation_account = self.impersonation_chain
elif len(self.impersonation_chain) == 1:
impersonation_account = self.impersonation_chain[0]
else:
raise AirflowException(
"Chained list of accounts is not supported, please specify only one service account"
)
gcp_options.append(f"--impersonate-service-account={impersonation_account}")
cmd = [
"gcloud",
"dataflow",
"sql",
"query",
query,
*gcp_options,
*(beam_options_to_args(options)),