This is an automated email from the ASF dual-hosted git repository.

tvalentyn pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new f7a9b26ab39 Allow yield and return in the same 'process' if return 
doesn't have a value. (#37190)
f7a9b26ab39 is described below

commit f7a9b26ab39a9250ed16e5ca2c72d6d4f98e8a6f
Author: tvalentyn <[email protected]>
AuthorDate: Mon Dec 29 10:27:34 2025 -0800

    Allow yield and return in the same 'process' if return doesn't have a 
value. (#37190)
---
 sdks/python/apache_beam/transforms/core.py      | 3 ++-
 sdks/python/apache_beam/transforms/core_test.py | 9 +++++++++
 2 files changed, 11 insertions(+), 1 deletion(-)

diff --git a/sdks/python/apache_beam/transforms/core.py 
b/sdks/python/apache_beam/transforms/core.py
index 74773a4d7ca..ea11bca9474 100644
--- a/sdks/python/apache_beam/transforms/core.py
+++ b/sdks/python/apache_beam/transforms/core.py
@@ -1515,7 +1515,8 @@ def _check_fn_use_yield_and_return(fn):
           "yield("):
         has_yield = True
       elif lstripped_line.rstrip() == "return":
-        has_return = True
+        # Return is likely used to exit the function - ok to use with 'yield'.
+        pass
       elif lstripped_line.startswith("return ") or lstripped_line.startswith(
           "return("):
         if lstripped_line.rstrip() == "return None" or lstripped_line.rstrip(
diff --git a/sdks/python/apache_beam/transforms/core_test.py 
b/sdks/python/apache_beam/transforms/core_test.py
index 80ab6a88afb..73f004c130c 100644
--- a/sdks/python/apache_beam/transforms/core_test.py
+++ b/sdks/python/apache_beam/transforms/core_test.py
@@ -43,6 +43,14 @@ from apache_beam.typehints import typehints
 RETURN_NONE_PARTIAL_WARNING = "Process method returned None"
 
 
+class TestDoFn0(beam.DoFn):
+  """Returning without a value is allowed"""
+  def process(self, element):
+    if not element:
+      return
+    yield element
+
+
 class TestDoFn1(beam.DoFn):
   def process(self, element):
     yield element
@@ -174,6 +182,7 @@ class CreateTest(unittest.TestCase):
 
     with self._caplog.at_level(logging.WARNING):
       assert beam.ParDo(sum)
+      assert beam.ParDo(TestDoFn0())
       assert beam.ParDo(TestDoFn1())
       assert beam.ParDo(TestDoFn2())
       assert beam.ParDo(TestDoFn4())

Reply via email to