[ 
https://issues.apache.org/jira/browse/BEAM-14014?focusedWorklogId=767902&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-767902
 ]

ASF GitHub Bot logged work on BEAM-14014:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 09/May/22 13:11
            Start Date: 09/May/22 13:11
    Worklog Time Spent: 10m 
      Work Description: kennknowles commented on code in PR #17244:
URL: https://github.com/apache/beam/pull/17244#discussion_r867993656


##########
sdks/python/apache_beam/io/gcp/gcsfilesystem.py:
##########
@@ -38,6 +40,15 @@ class GCSFileSystem(FileSystem):
   CHUNK_SIZE = gcsio.MAX_BATCH_OPERATION_SIZE  # Chuck size in batch operations
   GCS_PREFIX = 'gs://'
 
+  def __init__(self, pipeline_options):
+    super().__init__(pipeline_options)
+    if isinstance(pipeline_options, PipelineOptions):
+      gcs_options = pipeline_options.view_as(GoogleCloudOptions)
+      impersonate_service_account = gcs_options.impersonate_service_account

Review Comment:
   I would suggest not putting specific impersonation logic in `gcsfilesystem`. 
I think what you want is some place to put a shared 
`get_credentials(gcp_options)` method that will handle doing whatever is 
necessary. Then the gcs filesystem and the dataflow runner can both use this. 
You just want to be careful that impersonation is used during job submission 
and not used during job execution. In practice for Java I did this by having 
them in the pipeline options but having them disappear when serialized and 
shipped to workers.



##########
sdks/python/apache_beam/runners/dataflow/internal/apiclient.py:
##########
@@ -557,6 +558,8 @@ def __init__(self, options, root_staging_location=None):
     if self.google_cloud_options.no_auth:
       credentials = None
     else:
+      set_impersonation_accounts(
+          self.google_cloud_options.impersonate_service_account)

Review Comment:
   Around here would be the other use of `get_credentials(gcp_options)`. 
Perhaps that already lives in `get_service_credentials`. And around here is 
where you would produce the modified set of pipeline options for the job 
submission. I presume they are pickled into the job somewhere hereabouts?





Issue Time Tracking
-------------------

    Worklog Id:     (was: 767902)
    Time Spent: 3.5h  (was: 3h 20m)

> Support impersonation credentials in Dataflow runner.
> -----------------------------------------------------
>
>                 Key: BEAM-14014
>                 URL: https://issues.apache.org/jira/browse/BEAM-14014
>             Project: Beam
>          Issue Type: Improvement
>          Components: runner-dataflow
>            Reporter: Valentyn Tymofieiev
>            Assignee: Ryan Thompson
>            Priority: P2
>          Time Spent: 3.5h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to