-
Notifications
You must be signed in to change notification settings - Fork 4.4k
Description
What happened?
The --impersonate_service_account
option allows a principal to submit Dataflow jobs on behalf of another account in a delegation chain. This option is used when creating a GCP credential at job submission:
if impersonate_service_account: |
beam/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py
Lines 278 to 285 in 121ac71
# Don't pass impersonate_service_account through to the harness. | |
# Though impersonation should start a job, the workers should | |
# not try to modify their credentials. | |
options_dict.pop('impersonate_service_account', None) | |
self.proto.sdkPipelineOptions.additionalProperties.append( | |
dataflow.Environment.SdkPipelineOptionsValue.AdditionalProperty( | |
key='options', value=to_json_value(options_dict))) | |
However some Beam IOs might store a copy of a pipeline options, which contains the impersonation credential. When a pipeline with such IOs becomes serialzed in the Runner API representation, and the IO DoFns deserialize on the runner, we might accidentally capture the --impersonate_service_account
pipeline option and incorrectly use it at runtime.
When this happens, the worker logs might have a line 'Impersonating <... service account name ...> " at runtime. Such logs should only be applicable at job submission.
Note that creating an impersonated credential might be necessary at job submission for other purposes than submitting a dataflow job, for example to do pre-submission validations of GCP resources. To this end the PR #26662 created a different mechanism to authenticate a BQ client. Unfortunately, it inadvertently caused pipelines using BQ IO to incorrectly execute the impersonation flow at runtime, starting from Apache Beam Python 2.49.0 SDK.
Workaround
To work around the issue until Beam 2.59.0 becomes available, add the following code to the beginning of the pipeline launcher:
from apache_beam.options import pipeline_options
def getstate_patch(self):
# The impersonate_service_account option must be used only at submission of
# a Beam job. However, Beam IOs might store pipeline options
# within transform implementation that becomes serialized in RunnerAPI,
# causing this option to be inadvertently used at runtime.
# This serialization hook removes it.
if self.view_as(GoogleCloudOptions).impersonate_service_account:
dict_copy = dict(self.__dict__)
dict_copy['_all_options'] = dict(dict_copy['_all_options'])
dict_copy['_all_options']['impersonate_service_account'] = None
return dict_copy
else:
return self.__dict__
# Remove this after upgrading to Beam 2.59.0 or newer.
pipeline_options.PipelineOptions.__getstate__ = getstate_patch
<rest of the pipeline launcher code>
Issue Priority
Priority: 2 (default / most bugs should be filed as P2)
Issue Components
- Component: Python SDK
- Component: Java SDK
- Component: Go SDK
- Component: Typescript SDK
- Component: IO connector
- Component: Beam YAML
- Component: Beam examples
- Component: Beam playground
- Component: Beam katas
- Component: Website
- Component: Infrastructure
- Component: Spark Runner
- Component: Flink Runner
- Component: Samza Runner
- Component: Twister2 Runner
- Component: Hazelcast Jet Runner
- Component: Google Cloud Dataflow Runner