tvalentyn commented on code in PR #25743:
URL: https://github.com/apache/beam/pull/25743#discussion_r1134740493
##########
sdks/python/apache_beam/transforms/core.py:
##########
@@ -1427,6 +1478,16 @@ def __init__(self, fn, *args, **kwargs):
if not isinstance(self.fn, DoFn):
raise TypeError('ParDo must be called with a DoFn instance.')
+ # DoFn.process cannot allow both return and yield
+ if _check_fn_use_yield_and_return(self.fn.process):
+ _LOGGER.warning(
+ 'The yield and return statements in the process method '
Review Comment:
I would rephrase the beginning as: `Using yield and return in the process
method can lead to unexpected behavior, see:
https://github.com/apache/beam/issues/22969.`
##########
sdks/python/apache_beam/transforms/core.py:
##########
@@ -1387,6 +1388,56 @@ def partition_for(self, element, num_partitions, *args,
**kwargs):
return self._fn(element, num_partitions, *args, **kwargs)
+def _get_function_body_without_inners(func):
+ source_lines = inspect.getsourcelines(func)[0]
+ source_lines = dropwhile(lambda x: x.startswith("@"), source_lines)
+ def_line = next(source_lines).strip()
+ if def_line.startswith("def ") and def_line.endswith(":"):
+ first_line = next(source_lines)
+ indentation = len(first_line) - len(first_line.lstrip())
+ final_lines = [first_line[indentation:]]
+
+ skip_inner_def = False
+ if first_line[indentation:].startswith("def "):
+ skip_inner_def = True
+ for line in source_lines:
+ line_indentation = len(line) - len(line.lstrip())
+
+ if line[indentation:].startswith("def "):
+ skip_inner_def = True
+ continue
+
+ if skip_inner_def and line_indentation == indentation:
+ skip_inner_def = False
+
+ if skip_inner_def and line_indentation > indentation:
+ continue
+ final_lines.append(line[indentation:])
+
+ return "".join(final_lines)
+ else:
+ return def_line.rsplit(":")[-1].strip()
+
+
+def _check_fn_use_yield_and_return(fn):
+ if isinstance(fn, types.BuiltinFunctionType):
+ return False
+ try:
+ source_code = _get_function_body_without_inners(fn)
+ has_yield = False
+ has_return = False
+ for line in source_code.split("\n"):
+ if line.lstrip().startswith("yield"):
Review Comment:
this can still trigger false-positives for variables like yield_elements and
return_value (both occur in Beam).
consider checking for `startswith("yield(") or startswith("yield ") or
startswith("return(") or startswith("return ")`
##########
sdks/python/apache_beam/transforms/core.py:
##########
@@ -1387,6 +1388,47 @@ def partition_for(self, element, num_partitions, *args,
**kwargs):
return self._fn(element, num_partitions, *args, **kwargs)
+def _get_function_body_without_inners(func):
+ source_lines = inspect.getsourcelines(func)[0]
Review Comment:
> The doc was updated by #25060
we could recommend yield from instead of return for iterables, after
checking that yield from works well.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]