robertwb commented on code in PR #31458:
URL: https://github.com/apache/beam/pull/31458#discussion_r1625020210


##########
sdks/python/apache_beam/runners/common.py:
##########
@@ -564,13 +600,21 @@ def invoke_setup(self):
 
     """Invokes the DoFn.setup() method
     """
+    self._setup_context_values = {
+        d: d.context_manager.__enter__()

Review Comment:
   Good call. Done.



##########
sdks/python/apache_beam/runners/common.py:
##########
@@ -430,6 +430,42 @@ def has_bundle_finalization(self):
           pass
     return False
 
+  def get_bundle_contexts(self):
+    seen = set()
+    for sig in (self.setup_lifecycle_method,

Review Comment:
   Yeah, I don't think there's a good automated way to check that. 



##########
sdks/python/apache_beam/transforms/core.py:
##########
@@ -525,6 +525,44 @@ def __init__(
     self.param_id = 'WatermarkEstimatorProvider'
 
 
+class _BundleContextParam(_DoFnParam):
+  """Allows one to use a context manager to manage bundle-scoped parameters.
+
+  The context will be entered at the start of each bundle and exited at the
+  end, equivalent to the `start_bundle` and `finish_bundle` methods on a DoFn.
+
+  The object returned from `__enter__`, if any will be substituted for this
+  parameter in invocations.

Review Comment:
   Done.



##########
website/www/site/content/en/documentation/transforms/python/elementwise/pardo.md:
##########
@@ -76,10 +76,12 @@ starts and finishes with `start_bundle` and `finish_bundle`.
   multiple instances of a given `DoFn` subclass may be created (e.g., due to 
parallelization, or due to garbage collection after a period
   of disuse).
   This is a good place to connect to database instances, open network 
connections or other resources.
+  See also `DoFn.SetupContextParam` for a way to accomplish this via context 
managers.

Review Comment:
   Good point. Added to that list. I'm not sure how to improve the 
documentation of these parameters, but I'm adding an example to the elementwise 
map docs as you suggested so hopefully this will at least show up if you search 
for the term.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscr...@beam.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to