Skip to content

Commit f7a9b26

Browse files
authored
Allow yield and return in the same 'process' if return doesn't have a value. (#37190)
1 parent 9c4dc4d commit f7a9b26

File tree

2 files changed

+11
-1
lines changed

2 files changed

+11
-1
lines changed

sdks/python/apache_beam/transforms/core.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1515,7 +1515,8 @@ def _check_fn_use_yield_and_return(fn):
15151515
"yield("):
15161516
has_yield = True
15171517
elif lstripped_line.rstrip() == "return":
1518-
has_return = True
1518+
# Return is likely used to exit the function - ok to use with 'yield'.
1519+
pass
15191520
elif lstripped_line.startswith("return ") or lstripped_line.startswith(
15201521
"return("):
15211522
if lstripped_line.rstrip() == "return None" or lstripped_line.rstrip(

sdks/python/apache_beam/transforms/core_test.py

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,14 @@
4343
RETURN_NONE_PARTIAL_WARNING = "Process method returned None"
4444

4545

46+
class TestDoFn0(beam.DoFn):
47+
"""Returning without a value is allowed"""
48+
def process(self, element):
49+
if not element:
50+
return
51+
yield element
52+
53+
4654
class TestDoFn1(beam.DoFn):
4755
def process(self, element):
4856
yield element
@@ -174,6 +182,7 @@ def test_dofn_with_yield_and_return(self):
174182

175183
with self._caplog.at_level(logging.WARNING):
176184
assert beam.ParDo(sum)
185+
assert beam.ParDo(TestDoFn0())
177186
assert beam.ParDo(TestDoFn1())
178187
assert beam.ParDo(TestDoFn2())
179188
assert beam.ParDo(TestDoFn4())

0 commit comments

Comments
 (0)