35
35
from airflow .exceptions import AirflowException
36
36
from airflow .models import BaseOperator
37
37
from airflow .providers .google .cloud .hooks .gcs import GCSHook
38
+ from airflow .providers .google .common .links .storage import FileDetailsLink , StorageLink
38
39
from airflow .utils import timezone
39
40
40
41
@@ -107,6 +108,7 @@ class GCSCreateBucketOperator(BaseOperator):
107
108
'impersonation_chain' ,
108
109
)
109
110
ui_color = '#f0eee4'
111
+ operator_extra_links = (StorageLink (),)
110
112
111
113
def __init__ (
112
114
self ,
@@ -139,6 +141,12 @@ def execute(self, context: "Context") -> None:
139
141
delegate_to = self .delegate_to ,
140
142
impersonation_chain = self .impersonation_chain ,
141
143
)
144
+ StorageLink .persist (
145
+ context = context ,
146
+ task_instance = self ,
147
+ uri = self .bucket_name ,
148
+ project_id = self .project_id or hook .project_id ,
149
+ )
142
150
try :
143
151
hook .create_bucket (
144
152
bucket_name = self .bucket_name ,
@@ -200,6 +208,8 @@ class GCSListObjectsOperator(BaseOperator):
200
208
201
209
ui_color = '#f0eee4'
202
210
211
+ operator_extra_links = (StorageLink (),)
212
+
203
213
def __init__ (
204
214
self ,
205
215
* ,
@@ -234,6 +244,13 @@ def execute(self, context: "Context") -> list:
234
244
self .prefix ,
235
245
)
236
246
247
+ StorageLink .persist (
248
+ context = context ,
249
+ task_instance = self ,
250
+ uri = self .bucket ,
251
+ project_id = hook .project_id ,
252
+ )
253
+
237
254
return hook .list (bucket_name = self .bucket , prefix = self .prefix , delimiter = self .delimiter )
238
255
239
256
@@ -346,6 +363,7 @@ class GCSBucketCreateAclEntryOperator(BaseOperator):
346
363
'impersonation_chain' ,
347
364
)
348
365
# [END gcs_bucket_create_acl_template_fields]
366
+ operator_extra_links = (StorageLink (),)
349
367
350
368
def __init__ (
351
369
self ,
@@ -371,6 +389,12 @@ def execute(self, context: "Context") -> None:
371
389
gcp_conn_id = self .gcp_conn_id ,
372
390
impersonation_chain = self .impersonation_chain ,
373
391
)
392
+ StorageLink .persist (
393
+ context = context ,
394
+ task_instance = self ,
395
+ uri = self .bucket ,
396
+ project_id = hook .project_id ,
397
+ )
374
398
hook .insert_bucket_acl (
375
399
bucket_name = self .bucket , entity = self .entity , role = self .role , user_project = self .user_project
376
400
)
@@ -418,6 +442,7 @@ class GCSObjectCreateAclEntryOperator(BaseOperator):
418
442
'impersonation_chain' ,
419
443
)
420
444
# [END gcs_object_create_acl_template_fields]
445
+ operator_extra_links = (FileDetailsLink (),)
421
446
422
447
def __init__ (
423
448
self ,
@@ -447,6 +472,12 @@ def execute(self, context: "Context") -> None:
447
472
gcp_conn_id = self .gcp_conn_id ,
448
473
impersonation_chain = self .impersonation_chain ,
449
474
)
475
+ FileDetailsLink .persist (
476
+ context = context ,
477
+ task_instance = self ,
478
+ uri = f"{ self .bucket } /{ self .object_name } " ,
479
+ project_id = hook .project_id ,
480
+ )
450
481
hook .insert_object_acl (
451
482
bucket_name = self .bucket ,
452
483
object_name = self .object_name ,
@@ -498,6 +529,7 @@ class GCSFileTransformOperator(BaseOperator):
498
529
'transform_script' ,
499
530
'impersonation_chain' ,
500
531
)
532
+ operator_extra_links = (FileDetailsLink (),)
501
533
502
534
def __init__ (
503
535
self ,
@@ -549,6 +581,12 @@ def execute(self, context: "Context") -> None:
549
581
self .log .info ("Transformation succeeded. Output temporarily located at %s" , destination_file .name )
550
582
551
583
self .log .info ("Uploading file to %s as %s" , self .destination_bucket , self .destination_object )
584
+ FileDetailsLink .persist (
585
+ context = context ,
586
+ task_instance = self ,
587
+ uri = f"{ self .destination_bucket } /{ self .destination_object } " ,
588
+ project_id = hook .project_id ,
589
+ )
552
590
hook .upload (
553
591
bucket_name = self .destination_bucket ,
554
592
object_name = self .destination_object ,
@@ -628,6 +666,7 @@ class GCSTimeSpanFileTransformOperator(BaseOperator):
628
666
'source_impersonation_chain' ,
629
667
'destination_impersonation_chain' ,
630
668
)
669
+ operator_extra_links = (StorageLink (),)
631
670
632
671
@staticmethod
633
672
def interpolate_prefix (prefix : str , dt : datetime .datetime ) -> Optional [str ]:
@@ -718,6 +757,12 @@ def execute(self, context: "Context") -> List[str]:
718
757
gcp_conn_id = self .destination_gcp_conn_id ,
719
758
impersonation_chain = self .destination_impersonation_chain ,
720
759
)
760
+ StorageLink .persist (
761
+ context = context ,
762
+ task_instance = self ,
763
+ uri = self .destination_bucket ,
764
+ project_id = destination_hook .project_id ,
765
+ )
721
766
722
767
# Fetch list of files.
723
768
blobs_to_transform = source_hook .list_by_timespan (
@@ -904,6 +949,7 @@ class GCSSynchronizeBucketsOperator(BaseOperator):
904
949
'delegate_to' ,
905
950
'impersonation_chain' ,
906
951
)
952
+ operator_extra_links = (StorageLink (),)
907
953
908
954
def __init__ (
909
955
self ,
@@ -938,6 +984,12 @@ def execute(self, context: "Context") -> None:
938
984
delegate_to = self .delegate_to ,
939
985
impersonation_chain = self .impersonation_chain ,
940
986
)
987
+ StorageLink .persist (
988
+ context = context ,
989
+ task_instance = self ,
990
+ uri = self ._get_uri (self .destination_bucket , self .destination_object ),
991
+ project_id = hook .project_id ,
992
+ )
941
993
hook .sync (
942
994
source_bucket = self .source_bucket ,
943
995
destination_bucket = self .destination_bucket ,
@@ -947,3 +999,8 @@ def execute(self, context: "Context") -> None:
947
999
delete_extra_files = self .delete_extra_files ,
948
1000
allow_overwrite = self .allow_overwrite ,
949
1001
)
1002
+
1003
+ def _get_uri (self , gcs_bucket : str , gcs_object : Optional [str ]) -> str :
1004
+ if gcs_object and gcs_object [- 1 ] == "/" :
1005
+ gcs_object = gcs_object [:- 1 ]
1006
+ return f"{ gcs_bucket } /{ gcs_object } " if gcs_object else gcs_bucket
0 commit comments