You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ro...@apache.org on 2017/08/24 22:42:15 UTC

[3/3] beam git commit: Add URN registration mechanism for coders.

Add URN registration mechanism for coders.


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/ef4239ab
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/ef4239ab
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/ef4239ab

Branch: refs/heads/master
Commit: ef4239ab7928bfad95a4debb1517c2547473bf8f
Parents: d261d6b
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Tue Aug 22 10:01:40 2017 -0700
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Thu Aug 24 15:41:46 2017 -0700

----------------------------------------------------------------------
 sdks/python/apache_beam/coders/coders.py | 66 +++++++++++++++++++++------
 1 file changed, 53 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/ef4239ab/sdks/python/apache_beam/coders/coders.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py
index 7ced5a9..0ea5f7c 100644
--- a/sdks/python/apache_beam/coders/coders.py
+++ b/sdks/python/apache_beam/coders/coders.py
@@ -202,25 +202,65 @@ class Coder(object):
             and self._dict_without_impl() == other._dict_without_impl())
     # pylint: enable=protected-access
 
-  def to_runner_api(self, context):
-    """For internal use only; no backwards-compatibility guarantees.
+  _known_urns = {}
+
+  @classmethod
+  def register_urn(cls, urn, parameter_type, fn=None):
+    """Registeres a urn with a constructor.
+
+    For example, if 'beam:fn:foo' had paramter type FooPayload, one could
+    write `RunnerApiFn.register_urn('bean:fn:foo', FooPayload, foo_from_proto)`
+    where foo_from_proto took as arguments a FooPayload and a PipelineContext.
+    This function can also be used as a decorator rather than passing the
+    callable in as the final parameter.
+
+    A corresponding to_runner_api_parameter method would be expected that
+    returns the tuple ('beam:fn:foo', FooPayload)
     """
-    # TODO(BEAM-115): Use specialized URNs and components.
-    serialized_coder = serialize_coder(self)
+    def register(fn):
+      cls._known_urns[urn] = parameter_type, fn
+      return staticmethod(fn)
+    if fn:
+      # Used as a statement.
+      register(fn)
+    else:
+      # Used as a decorator.
+      return register
+
+  def to_runner_api(self, context):
+    from apache_beam.portability.api import beam_runner_api_pb2
+    urn, typed_param, components = self.to_runner_api_parameter(context)
     return beam_runner_api_pb2.Coder(
         spec=beam_runner_api_pb2.SdkFunctionSpec(
             spec=beam_runner_api_pb2.FunctionSpec(
-                urn=urns.PICKLED_CODER,
-                any_param=proto_utils.pack_Any(
-                    google.protobuf.wrappers_pb2.BytesValue(
-                        value=serialized_coder)),
-                payload=serialized_coder)))
+                urn=urn,
+                any_param=proto_utils.pack_Any(typed_param),
+                payload=typed_param.SerializeToString()
+                if typed_param is not None else None)),
+        component_coder_ids=[context.coders.get_id(c) for c in components])
 
-  @staticmethod
-  def from_runner_api(proto, context):
-    """For internal use only; no backwards-compatibility guarantees.
+  @classmethod
+  def from_runner_api(cls, coder_proto, context):
+    """Converts from an SdkFunctionSpec to a Fn object.
+
+    Prefer registering a urn with its parameter type and constructor.
     """
-    return deserialize_coder(proto.spec.spec.payload)
+    parameter_type, constructor = cls._known_urns[coder_proto.spec.spec.urn]
+    return constructor(
+        proto_utils.parse_Bytes(coder_proto.spec.spec.payload, parameter_type),
+        [context.coders.get_by_id(c) for c in coder_proto.component_coder_ids],
+        context)
+
+  def to_runner_api_parameter(self, context):
+    return (
+        urns.PICKLED_CODER,
+        google.protobuf.wrappers_pb2.BytesValue(value=serialize_coder(self)),
+        ())
+
+
+@Coder.register_urn(urns.PICKLED_CODER, google.protobuf.wrappers_pb2.BytesValue)
+def _pickle_from_runner_api_parameter(payload, components, context):
+  return deserialize_coder(payload.value)
 
 
 class StrUtf8Coder(Coder):