You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by al...@apache.org on 2018/10/31 18:49:24 UTC
[beam] branch master updated: [BEAM-5878] [BEAM-5490] Add partial
support for functions with keyword-only arguments. (#6781)
This is an automated email from the ASF dual-hosted git repository.
altay 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 aab2f75 [BEAM-5878] [BEAM-5490] Add partial support for functions with keyword-only arguments. (#6781)
aab2f75 is described below
commit aab2f75e7c89d48e324c17dddefab31d2d93fbb6
Author: tvalentyn <tv...@users.noreply.github.com>
AuthorDate: Wed Oct 31 11:49:17 2018 -0700
[BEAM-5878] [BEAM-5490] Add partial support for functions with keyword-only arguments. (#6781)
* [BEAM-5878] [BEAM-5490] Add partial support for functions with keyword-only arguments: use getfullargspec instead of getargspec on Python 3, which recognizes kwonlyargs, avoid an irreverisble monkey-patch of inspect.getargspec.
---
sdks/python/apache_beam/runners/common.py | 7 ++-
sdks/python/apache_beam/transforms/core.py | 21 +++++--
sdks/python/apache_beam/transforms/ptransform.py | 6 +-
sdks/python/apache_beam/typehints/decorators.py | 64 ++++++++++++++++------
.../apache_beam/typehints/typed_pipeline_test.py | 4 +-
.../python/apache_beam/typehints/typehints_test.py | 3 +-
6 files changed, 75 insertions(+), 30 deletions(-)
diff --git a/sdks/python/apache_beam/runners/common.py b/sdks/python/apache_beam/runners/common.py
index 47222a1..0336500 100644
--- a/sdks/python/apache_beam/runners/common.py
+++ b/sdks/python/apache_beam/runners/common.py
@@ -150,8 +150,13 @@ class MethodWrapper(object):
'a \'RestrictionProvider\'. Received %r instead.'
% obj_to_invoke)
- args, _, _, defaults = core.get_function_arguments(
+ fullargspec = core.get_function_arguments(
obj_to_invoke, method_name)
+
+ # TODO(BEAM-5878) support kwonlyargs on Python 3.
+ args = fullargspec[0]
+ defaults = fullargspec[3]
+
defaults = defaults if defaults else []
method_value = getattr(obj_to_invoke, method_name)
self.method_value = method_value
diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py
index 5505de5..9027b97 100644
--- a/sdks/python/apache_beam/transforms/core.py
+++ b/sdks/python/apache_beam/transforms/core.py
@@ -20,7 +20,6 @@
from __future__ import absolute_import
import copy
-import inspect
import logging
import random
import re
@@ -64,6 +63,7 @@ from apache_beam.typehints import trivial_inference
from apache_beam.typehints.decorators import TypeCheckError
from apache_beam.typehints.decorators import WithTypeHints
from apache_beam.typehints.decorators import get_type_hints
+from apache_beam.typehints.decorators import getfullargspec
from apache_beam.typehints.trivial_inference import element_type
from apache_beam.typehints.typehints import is_consistent_with
from apache_beam.utils import urns
@@ -275,14 +275,14 @@ class RestrictionProvider(object):
def get_function_arguments(obj, func):
"""Return the function arguments based on the name provided. If they have
a _inspect_function attached to the class then use that otherwise default
- to the python inspect library.
+ to the modified version of python inspect library.
"""
func_name = '_inspect_%s' % func
if hasattr(obj, func_name):
f = getattr(obj, func_name)
return f()
f = getattr(obj, func)
- return inspect.getargspec(f)
+ return getfullargspec(f)
class _DoFnParam(object):
@@ -442,12 +442,21 @@ class DoFn(WithTypeHints, HasDisplayData, urns.RunnerApiFn):
def _fn_takes_side_inputs(fn):
try:
- argspec = inspect.getargspec(fn)
+ argspec = getfullargspec(fn)
except TypeError:
# We can't tell; maybe it does.
return True
is_bound = isinstance(fn, types.MethodType) and fn.__self__ is not None
- return len(argspec.args) > 1 + is_bound or argspec.varargs or argspec.keywords
+
+ try:
+ varkw = argspec.varkw
+ kwonlyargs = argspec.kwonlyargs
+ except AttributeError: # Python 2
+ varkw = argspec.keywords
+ kwonlyargs = []
+
+ return (len(argspec.args) + len(kwonlyargs) > 1 + is_bound or
+ argspec.varargs or varkw)
class CallableWrapperDoFn(DoFn):
@@ -515,7 +524,7 @@ class CallableWrapperDoFn(DoFn):
return getattr(self._fn, '_argspec_fn', self._fn)
def _inspect_process(self):
- return inspect.getargspec(self._process_argspec_fn())
+ return getfullargspec(self._process_argspec_fn())
class CombineFn(WithTypeHints, HasDisplayData, urns.RunnerApiFn):
diff --git a/sdks/python/apache_beam/transforms/ptransform.py b/sdks/python/apache_beam/transforms/ptransform.py
index 9939278..c512d9f 100644
--- a/sdks/python/apache_beam/transforms/ptransform.py
+++ b/sdks/python/apache_beam/transforms/ptransform.py
@@ -37,7 +37,6 @@ FlatMap processing functions.
from __future__ import absolute_import
import copy
-import inspect
import itertools
import operator
import os
@@ -61,6 +60,7 @@ from apache_beam.typehints import typehints
from apache_beam.typehints.decorators import TypeCheckError
from apache_beam.typehints.decorators import WithTypeHints
from apache_beam.typehints.decorators import getcallargs_forhints
+from apache_beam.typehints.decorators import getfullargspec
from apache_beam.typehints.trivial_inference import instance_to_type
from apache_beam.typehints.typehints import validate_composite_type_param
from apache_beam.utils import proto_utils
@@ -753,8 +753,10 @@ class _PTransformFnPTransform(PTransform):
# .with_output_types() methods.
kwargs = dict(self._kwargs)
args = tuple(self._args)
+
+ # TODO(BEAM-5878) Support keyword-only arguments.
try:
- if 'type_hints' in inspect.getargspec(self._fn).args:
+ if 'type_hints' in getfullargspec(self._fn).args:
args = (self.get_type_hints(),) + args
except TypeError:
# Might not be a function.
diff --git a/sdks/python/apache_beam/typehints/decorators.py b/sdks/python/apache_beam/typehints/decorators.py
index 6604cf1..7d2441e 100644
--- a/sdks/python/apache_beam/typehints/decorators.py
+++ b/sdks/python/apache_beam/typehints/decorators.py
@@ -112,38 +112,42 @@ __all__ = [
_MethodDescriptorType = type(str.upper)
# pylint: enable=invalid-name
+try:
+ _original_getfullargspec = inspect.getfullargspec
+ _use_full_argspec = True
+except AttributeError: # Python 2
+ _original_getfullargspec = inspect.getargspec
+ _use_full_argspec = False
-# Monkeypatch inspect.getargspec to allow passing non-function objects.
-# This is needed to use higher-level functions such as getcallargs.
-_original_getargspec = inspect.getargspec
-
-def getargspec(func):
+def getfullargspec(func):
try:
- return _original_getargspec(func)
+ return _original_getfullargspec(func)
except TypeError:
if isinstance(func, type):
- argspec = getargspec(func.__init__)
+ argspec = getfullargspec(func.__init__)
del argspec.args[0]
return argspec
elif callable(func):
try:
- return _original_getargspec(func.__call__)
+ return _original_getfullargspec(func.__call__)
except TypeError:
# Return an ArgSpec with at least one positional argument,
# and any number of other (positional or keyword) arguments
- # whose name won't match any real agument.
+ # whose name won't match any real argument.
# Arguments with the %unknown% prefix will be ignored in the type
# checking code.
- return inspect.ArgSpec(
- ['_'], '__unknown__varargs', '__unknown__keywords', ())
+ if _use_full_argspec:
+ return inspect.FullArgSpec(
+ ['_'], '__unknown__varargs', '__unknown__keywords', (),
+ [], {}, {})
+ else: # Python 2
+ return inspect.ArgSpec(
+ ['_'], '__unknown__varargs', '__unknown__keywords', ())
else:
raise
-inspect.getargspec = getargspec
-
-
class IOTypeHints(object):
"""Encapsulates all type hint information about a Dataflow construct.
@@ -259,15 +263,31 @@ def _unpack_positional_arg_hints(arg, hint):
def getcallargs_forhints(func, *typeargs, **typekwargs):
"""Like inspect.getcallargs, but understands that Tuple[] and an Any unpack.
"""
- argspec = inspect.getargspec(func)
+ argspec = getfullargspec(func)
# Turn Tuple[x, y] into (x, y) so getcallargs can do the proper unpacking.
packed_typeargs = [_unpack_positional_arg_hints(arg, hint)
for (arg, hint) in zip(argspec.args, typeargs)]
packed_typeargs += list(typeargs[len(packed_typeargs):])
+
+ # Monkeypatch inspect.getfullargspec to allow passing non-function objects.
+ # getfullargspec (getargspec on Python 2) are used by inspect.getcallargs.
+ # TODO(BEAM-5490): Reimplement getcallargs and stop relying on monkeypatch.
+ if _use_full_argspec:
+ inspect.getfullargspec = getfullargspec
+ else: # Python 2
+ inspect.getargspec = getfullargspec
+
try:
callargs = inspect.getcallargs(func, *packed_typeargs, **typekwargs)
except TypeError as e:
raise TypeCheckError(e)
+ finally:
+ # Revert monkey-patch.
+ if _use_full_argspec:
+ inspect.getfullargspec = _original_getfullargspec
+ else:
+ inspect.getargspec = _original_getfullargspec
+
if argspec.defaults:
# Declare any default arguments to be Any.
for k, var in enumerate(reversed(argspec.args)):
@@ -279,10 +299,18 @@ def getcallargs_forhints(func, *typeargs, **typekwargs):
if argspec.varargs:
callargs[argspec.varargs] = typekwargs.get(
argspec.varargs, typehints.Tuple[typehints.Any, ...])
- if argspec.keywords:
+ if _use_full_argspec:
+ varkw = argspec.varkw
+ else: # Python 2
+ varkw = argspec.keywords
+
+ if varkw:
# TODO(robertwb): Consider taking the union of key and value types.
- callargs[argspec.keywords] = typekwargs.get(
- argspec.keywords, typehints.Dict[typehints.Any, typehints.Any])
+ callargs[varkw] = typekwargs.get(
+ varkw, typehints.Dict[typehints.Any, typehints.Any])
+
+ # TODO(BEAM-5878) Support kwonlyargs.
+
return callargs
diff --git a/sdks/python/apache_beam/typehints/typed_pipeline_test.py b/sdks/python/apache_beam/typehints/typed_pipeline_test.py
index 22f58c0..b3335ef 100644
--- a/sdks/python/apache_beam/typehints/typed_pipeline_test.py
+++ b/sdks/python/apache_beam/typehints/typed_pipeline_test.py
@@ -19,7 +19,6 @@
from __future__ import absolute_import
-import inspect
import os
import sys
import typing
@@ -33,6 +32,7 @@ from apache_beam.testing.test_pipeline import TestPipeline
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to
from apache_beam.typehints import WithTypeHints
+from apache_beam.typehints.decorators import getfullargspec
# These test often construct a pipeline as value | PTransform to test side
# effects (e.g. errors).
@@ -162,7 +162,7 @@ class SideInputTest(unittest.TestCase):
['a', 'bb', 'c'] | beam.Map(repeat, times='z')
with self.assertRaises(typehints.TypeCheckError):
['a', 'bb', 'c'] | beam.Map(repeat, 3, 4)
- if not inspect.getargspec(repeat).defaults:
+ if not getfullargspec(repeat).defaults:
with self.assertRaises(typehints.TypeCheckError):
['a', 'bb', 'c'] | beam.Map(repeat)
diff --git a/sdks/python/apache_beam/typehints/typehints_test.py b/sdks/python/apache_beam/typehints/typehints_test.py
index e1106ce..f002cd0 100644
--- a/sdks/python/apache_beam/typehints/typehints_test.py
+++ b/sdks/python/apache_beam/typehints/typehints_test.py
@@ -38,6 +38,7 @@ from apache_beam.typehints.decorators import _interleave_type_check
from apache_beam.typehints.decorators import _positional_arg_hints
from apache_beam.typehints.decorators import get_type_hints
from apache_beam.typehints.decorators import getcallargs_forhints
+from apache_beam.typehints.decorators import getfullargspec
from apache_beam.typehints.typehints import is_consistent_with
@@ -66,7 +67,7 @@ def check_type_hints(f):
kwargs[var] = new_value
else:
args = list(args)
- for ix, pvar in enumerate(inspect.getargspec(f).args):
+ for ix, pvar in enumerate(getfullargspec(f).args):
if pvar == var:
args[ix] = new_value
break