Skip to content

Commit 1f63118

Browse files
authored
Fix warning message (#36703)
* Fix spammy warning message (and its formatting) * address comments - make raw return a valid return * address comments * add back tests
1 parent 3a1ae18 commit 1f63118

File tree

2 files changed

+20
-17
lines changed

2 files changed

+20
-17
lines changed

sdks/python/apache_beam/transforms/core.py

Lines changed: 12 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -1507,25 +1507,28 @@ def _check_fn_use_yield_and_return(fn):
15071507
source_code = _get_function_body_without_inners(fn)
15081508
has_yield = False
15091509
has_return = False
1510-
return_none_warning = (
1511-
"No iterator is returned by the process method in %s.",
1512-
fn.__self__.__class__)
1510+
has_return_none = False
15131511
for line in source_code.split("\n"):
15141512
lstripped_line = line.lstrip()
15151513
if lstripped_line.startswith("yield ") or lstripped_line.startswith(
15161514
"yield("):
15171515
has_yield = True
1518-
if lstripped_line.startswith("return ") or lstripped_line.startswith(
1516+
elif lstripped_line.rstrip() == "return":
1517+
has_return = True
1518+
elif lstripped_line.startswith("return ") or lstripped_line.startswith(
15191519
"return("):
1520+
if lstripped_line.rstrip() == "return None" or lstripped_line.rstrip(
1521+
) == "return(None)":
1522+
has_return_none = True
15201523
has_return = True
1521-
if lstripped_line.startswith(
1522-
"return None") or lstripped_line.rstrip() == "return":
1523-
_LOGGER.warning(return_none_warning)
15241524
if has_yield and has_return:
15251525
return True
15261526

1527-
if not has_yield and not has_return:
1528-
_LOGGER.warning(return_none_warning)
1527+
if has_return_none:
1528+
_LOGGER.warning(
1529+
"Process method returned None (element won't be emitted): %s."
1530+
" Check if intended.",
1531+
fn.__self__.__class__)
15291532

15301533
return False
15311534
except Exception as e:

sdks/python/apache_beam/transforms/core_test.py

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@
4040
from apache_beam.typehints import row_type
4141
from apache_beam.typehints import typehints
4242

43-
RETURN_NONE_PARTIAL_WARNING = "No iterator is returned"
43+
RETURN_NONE_PARTIAL_WARNING = "Process method returned None"
4444

4545

4646
class TestDoFn1(beam.DoFn):
@@ -121,9 +121,11 @@ def process(self, element):
121121

122122

123123
class TestDoFn12(beam.DoFn):
124-
"""test process returning None (return statement without a value)"""
124+
"""test process returning None in a filter pattern"""
125125
def process(self, element):
126-
return
126+
if element == 0:
127+
return
128+
return element
127129

128130

129131
class TestDoFnStateful(beam.DoFn):
@@ -194,14 +196,12 @@ def test_dofn_with_explicit_return_none(self):
194196
def test_dofn_with_implicit_return_none_missing_return_and_yield(self):
195197
with self._caplog.at_level(logging.WARNING):
196198
beam.ParDo(TestDoFn11())
197-
assert RETURN_NONE_PARTIAL_WARNING in self._caplog.text
198-
assert str(TestDoFn11) in self._caplog.text
199+
assert RETURN_NONE_PARTIAL_WARNING not in self._caplog.text
199200

200-
def test_dofn_with_implicit_return_none_return_without_value(self):
201+
def test_dofn_with_implicit_return_none_and_value(self):
201202
with self._caplog.at_level(logging.WARNING):
202203
beam.ParDo(TestDoFn12())
203-
assert RETURN_NONE_PARTIAL_WARNING in self._caplog.text
204-
assert str(TestDoFn12) in self._caplog.text
204+
assert RETURN_NONE_PARTIAL_WARNING not in self._caplog.text
205205

206206

207207
class PartitionTest(unittest.TestCase):

0 commit comments

Comments
 (0)