-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Add pipeline option to enforce gbek #36321
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
dfb53d0
41a0612
d9d02b1
8063593
04dae6c
f96dfba
1261ae2
d148cd2
ac4e467
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -341,6 +341,44 @@ def generate_secret_bytes() -> bytes: | |
| """Generates a new secret key.""" | ||
| return Fernet.generate_key() | ||
|
|
||
| @staticmethod | ||
| def parse_secret_option(secret) -> 'Secret': | ||
| """Parses a secret string and returns the appropriate secret type. | ||
|
|
||
| The secret string should be formatted like: | ||
| 'type:<secret_type>;<secret_param>:<value>' | ||
|
|
||
| For example, 'type:GcpSecret;version_name:my_secret/versions/latest' | ||
| would return a GcpSecret initialized with 'my_secret/versions/latest'. | ||
| """ | ||
| param_map = {} | ||
| for param in secret.split(';'): | ||
| parts = param.split(':') | ||
| param_map[parts[0]] = parts[1] | ||
damccorm marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| if 'type' not in param_map: | ||
| raise ValueError('Secret string must contain a valid type parameter') | ||
|
|
||
| secret_type = param_map['type'].lower() | ||
| del param_map['type'] | ||
| secret_class = None | ||
| secret_params = None | ||
| if secret_type == 'gcpsecret': | ||
| secret_class = GcpSecret | ||
| secret_params = ['version_name'] | ||
| else: | ||
| raise ValueError( | ||
| f'Invalid secret type {secret_type}, currently only ' | ||
| 'GcpSecret is supported') | ||
|
|
||
| for param_name in param_map.keys(): | ||
| if param_name not in secret_params: | ||
| raise ValueError( | ||
| f'Invalid secret parameter {param_name}, ' | ||
| f'{secret_type} only supports the following ' | ||
| f'parameters: {secret_params}') | ||
| return secret_class(**param_map) | ||
|
|
||
|
|
||
| class GcpSecret(Secret): | ||
| """A secret manager implementation that retrieves secrets from Google Cloud | ||
|
|
@@ -367,7 +405,12 @@ def get_secret_bytes(self) -> bytes: | |
| secret = response.payload.data | ||
| return secret | ||
| except Exception as e: | ||
| raise RuntimeError(f'Failed to retrieve secret bytes with excetion {e}') | ||
| raise RuntimeError( | ||
| 'Failed to retrieve secret bytes for secret ' | ||
| f'{self._version_name} with exception {e}') | ||
|
|
||
| def __eq__(self, secret): | ||
| return self._version_name == getattr(secret, '_version_name', None) | ||
|
|
||
|
|
||
| class _EncryptMessage(DoFn): | ||
|
|
@@ -499,7 +542,9 @@ def __init__(self, hmac_key: Secret): | |
| self._hmac_key = hmac_key | ||
|
|
||
| def expand(self, pcoll): | ||
| kv_type_hint = pcoll.element_type | ||
| key_type, value_type = (typehints.typehints.coerce_to_kv_type( | ||
| pcoll.element_type).tuple_types) | ||
| kv_type_hint = typehints.KV[key_type, value_type] | ||
| if kv_type_hint and kv_type_hint != typehints.Any: | ||
| coder = coders.registry.get_coder(kv_type_hint).as_deterministic_coder( | ||
| f'GroupByEncryptedKey {self.label}' | ||
|
|
@@ -518,10 +563,13 @@ def expand(self, pcoll): | |
| key_coder = coders.registry.get_coder(typehints.Any) | ||
| value_coder = key_coder | ||
|
|
||
| gbk = beam.GroupByKey() | ||
| gbk._inside_gbek = True | ||
|
|
||
| return ( | ||
| pcoll | ||
| | beam.ParDo(_EncryptMessage(self._hmac_key, key_coder, value_coder)) | ||
| | beam.GroupByKey() | ||
| | gbk | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I guess this breaks the update compatibility given the line number changes.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That's true - that's why I think we should do #36251 (comment) to permanently avoid this problem. I can follow up with a PR
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. #36381 will fix this broadly. |
||
| | beam.ParDo(_DecryptMessage(self._hmac_key, key_coder, value_coder))) | ||
|
|
||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
shall we somehow retry
get_secret_bytes?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This gets called during setup, so it should get automatic retries from the runner -
beam/sdks/python/apache_beam/transforms/util.py
Line 385 in a03e96a