-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Add GroupByEncryptedKey transform #36213
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
Conversation
|
There are some failing tests, but they all seem unrelated and the core gbek tests are succeeding. |
|
Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment |
|
assign set of reviewers |
|
Assigning reviewers: R: @claudevdm for label python. Note: If you would like to opt out of this review, comment Available commands:
The PR bot will only process comments in the main thread (not review comments). |
|
|
||
| try: | ||
| import dill | ||
| from google.cloud import secretmanager |
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.
Should we move this out of the dill try/except block? This import will always fail if dill is not installed.
Also related, in which test suite do you expect the tests that depend on secretmanager to run? I see they are skipped in PreCommit Python Coverage apache_beam/transforms/util_test.py:354: GCP dependencies are not installed
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.
Thanks, updated. It should run in I think, or at least in the postcommit. I'm not 100% sure where all we have GCP deps installed. I'll verify it runs correctly this time, though, thanks for flagging it.
| raise ValueError( | ||
| 'Input elements to the transform %s with stateful DoFn must be ' | ||
| 'key-value pairs.' % self) | ||
| key_coder = coder.key_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.
Should we assert that the key coder is deterministic here/ call self.key_coder.as_deterministic_coder here? For the encrypted key to be deterministic, we assume that the actual key coder is deterministic?
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.
Wondering if its possible to create a new coder that wraps the original coder along with doing the encryption?
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.
I think defining this as a coder could be messy because then you would need to access the secret at construction time and include that as part of the serialized graph definition. This would then not provide sufficient security guarantees since the graph itself would have all information needed to decrypt the value.
Potentially you could include the work of downloading the secret in the coder definition, but I don't think we gain much from this today (and errors might be messy to debug). It also seems nice that we have the actual transform definition which helps make it obvious what is happening from the graph.
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.
Yeah, this is a good call. Interestingly, we do check this for the direct runner GBK implementation, but not more broadly as far as I can tell. But we should definitely be verifying here.
Updated
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.
Got it. I think this will still circumvent update compat checks because we are pulling the original coders out of the graph?
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.
Yeah, it will, though at least the pipeline should cleanly fail in most cases when it tries to perform the encoding. For now, I will add a note to the doc string that this should be used with caution to avoid this issue.
In the future, there are maybe ways we could address this:
- Instead of baking this into a transform, we could create a custom coder at the SDK level which handles this (reading in the encoded bytes and then applying the encoding on top of that in bundle_processor.py)
- We could create a custom type and associated coder per GBEK instance which handles all encoding pieces.
Neither of these is trivial, so for now I will leave this with the doc note
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.
Something like the coder.encode could process a tuple (secret_version_name, unencrypted_key) and, use a output a (secret_version_name, encrypted_key)?
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.
I thought about this as well - it is pretty expensive though since we're serializing the secret name every time.
This is what I was trying to get around with We could create a custom type and associated coder per GBEK instance which handles all encoding pieces. since the coder definition could contain the secret name.
| # GCP Packages required by ML functionality | ||
| 'google-cloud-dlp>=3.0.0,<4', | ||
| 'google-cloud-language>=2.0,<3', | ||
| 'google-cloud-secret-manager>=2.0,<3', |
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.
I know Ashok had some issues with google-cloud-secret-manager dependency when using it as an additional requirement (because of all the google cloud library namespaces), but that might be resolved if the dependency is installed on the containers
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.
I haven't had any issues here - was it just that finding a version that matched requirements was hard?
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.
I think it was that requirements installed it in a different directory (site packages vs dist packages) than all the other gcp deps. This caused "scecretmanager not defined" issue because it was looking in the path of all the other gcp packages (they were in dist packages when secretmanager was installed in site packages or vice versa)
Looks like I need to give broader access to the service account, I'll update back once this is passing. |
|
waiting on author |
Fixed - tests are now passing |
Add a transform which allows users to use a secret manager to ensure that any GBK data is encrypted before being sent to a runner. This allows users to avoid unencrypted data at rest when using GBK.
Part of #36214
Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
addresses #123), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, commentfixes #<ISSUE NUMBER>instead.CHANGES.mdwith noteworthy changes.See the Contributor Guide for more tips on how to make review process smoother.
To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md
GitHub Actions Tests Status (on master branch)
See CI.md for more information about GitHub Actions CI or the workflows README to see a list of phrases to trigger workflows.