[ 
https://issues.apache.org/jira/browse/BEAM-7886?focusedWorklogId=324788&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-324788
 ]

ASF GitHub Bot logged work on BEAM-7886:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 08/Oct/19 00:43
            Start Date: 08/Oct/19 00:43
    Worklog Time Spent: 10m 
      Work Description: TheNeuralBit commented on pull request #9188: 
[BEAM-7886] Make row coder a standard coder and implement in Python
URL: https://github.com/apache/beam/pull/9188#discussion_r332295459
 
 

 ##########
 File path: sdks/python/apache_beam/coders/row_coder.py
 ##########
 @@ -0,0 +1,173 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+from __future__ import absolute_import
+
+import itertools
+from array import array
+
+from apache_beam.coders.coder_impl import StreamCoderImpl
+from apache_beam.coders.coders import BytesCoder
+from apache_beam.coders.coders import Coder
+from apache_beam.coders.coders import FastCoder
+from apache_beam.coders.coders import FloatCoder
+from apache_beam.coders.coders import IterableCoder
+from apache_beam.coders.coders import StrUtf8Coder
+from apache_beam.coders.coders import TupleCoder
+from apache_beam.coders.coders import VarIntCoder
+from apache_beam.portability import common_urns
+from apache_beam.portability.api import schema_pb2
+from apache_beam.typehints.schemas import named_tuple_from_schema
+from apache_beam.typehints.schemas import named_tuple_to_schema
+
+__all__ = ["RowCoder"]
+
+
+class RowCoder(FastCoder):
+  """ Coder for `typing.NamedTuple` instances.
+
+  Implements the beam:coder:row:v1 standard coder spec.
+  """
+
+  def __init__(self, schema):
+    """Initializes a :class:`RowCoder`.
+
+    Args:
+      schema (apache_beam.portability.api.schema_pb2.Schema): The protobuf
+        representation of the schema of the data that the RowCoder will be used
+        to encode/decode.
+    """
+    self.schema = schema
+    self.components = [
+        coder_from_type(field.type) for field in self.schema.fields
+    ]
+
+  def _create_impl(self):
+    return RowCoderImpl(self.schema, self.components)
+
+  def is_deterministic(self):
+    return all(c.is_deterministic() for c in self.components)
+
+  def to_type_hint(self):
+    return named_tuple_from_schema(self.schema)
+
+  def as_cloud_object(self, coders_context=None):
+    raise NotImplementedError("as_cloud_object not supported for RowCoder")
+
+  __hash__ = None
+
+  def __eq__(self, other):
+    return type(self) == type(other) and self.schema == other.schema
+
+  def to_runner_api_parameter(self, unused_context):
+    return (common_urns.coders.ROW.urn, self.schema, [])
+
+  @Coder.register_urn(common_urns.coders.ROW.urn, schema_pb2.Schema)
+  def from_runner_api_parameter(payload, components, unused_context):
+    return RowCoder(payload)
+
+  @staticmethod
+  def from_type_hint(named_tuple_type, registry):
+    return RowCoder(named_tuple_to_schema(named_tuple_type))
+
+
+def coder_from_type(type_):
+  type_info = type_.WhichOneof("type_info")
+  if type_info == "atomic_type":
+    if type_.atomic_type in (schema_pb2.INT32,
+                             schema_pb2.INT64):
+      return VarIntCoder()
+    elif type_.atomic_type == schema_pb2.DOUBLE:
+      return FloatCoder()
+    elif type_.atomic_type == schema_pb2.STRING:
+      return StrUtf8Coder()
+  elif type_info == "array_type":
+    return IterableCoder(coder_from_type(type_.array_type.element_type))
+
+  # The Java SDK supports several more types, but the coders are not yet
+  # standard, and are not implemented in Python.
+  raise ValueError(
+      "Encountered a type that is not currently supported by RowCoder: %s" %
+      type_)
+
+
+class RowCoderImpl(StreamCoderImpl):
+  """For internal use only; no backwards-compatibility guarantees."""
+  SIZE_CODER = VarIntCoder().get_impl()
+  NULL_CODER = BytesCoder().get_impl()
+
+  def __init__(self, schema, components):
+    self.schema = schema
+    self.constructor = named_tuple_from_schema(schema)
+    self.components = list(c.get_impl() for c in components)
+    self.has_nullable_fields = any(
+        field.type.nullable for field in self.schema.fields)
+
+  def encode_to_stream(self, value, out, nested):
+    nvals = len(self.schema.fields)
+    self.SIZE_CODER.encode_to_stream(nvals, out, True)
+    attrs = [getattr(value, f.name) for f in self.schema.fields]
+
+    words = array('B')
+    if self.has_nullable_fields:
+      nulls = list(attr is None for attr in attrs)
+      if any(nulls):
+        words = array('B', itertools.repeat(0, (nvals+7)//8))
+        for i, is_null in enumerate(nulls):
+          words[i//8] |= is_null << (i % 8)
+
+    self.NULL_CODER.encode_to_stream(words.tostring(), out, True)
+
+    for c, field, attr in zip(self.components, self.schema.fields, attrs):
+      if attr is None:
+        if not field.type.nullable:
+          raise ValueError(
+              "Attempted to encode null for non-nullable field \"{}\".".format(
+                  field.name))
+        continue
+      c.encode_to_stream(attr, out, True)
+
+  def decode_from_stream(self, in_stream, nested):
+    nvals = self.SIZE_CODER.decode_from_stream(in_stream, True)
+    words = array('B')
+    words.fromstring(self.NULL_CODER.decode_from_stream(in_stream, True))
+
+    if words:
+      nulls = ((words[i // 8] >> (i % 8)) & 0x01 for i in range(nvals))
+    else:
+      nulls = itertools.repeat(False, nvals)
+
+    # If this coder's schema has more attributes than the encoded value, then
+    # the schema must have changed. Populate the unencoded fields with nulls.
+    if len(self.components) > nvals:
+      nulls = itertools.chain(
+          nulls,
+          itertools.repeat(True, len(self.components) - nvals))
+
+    # Note that if this coder's schema has *fewer* attributes than the encoded
+    # value, we just need to ignore the additional values, which will occur
+    # here because we only decode as many values as we have coders for.
 
 Review comment:
   That's a good point.. I was just trying to replicate what is implemented in 
[java's 
RowCoderGenerator](https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoderGenerator.java#L308).
 It seems like this would be an issue over there as well, unless something else 
is consuming the unread bytes? Could there be logic to do that when there's a 
length-prefix?
   
   I'd be fine with just leaving this out for now and filing a jira if we can't 
get a satisfying answer.
 
----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 324788)
    Time Spent: 12h  (was: 11h 50m)

> Make row coder a standard coder and implement in python
> -------------------------------------------------------
>
>                 Key: BEAM-7886
>                 URL: https://issues.apache.org/jira/browse/BEAM-7886
>             Project: Beam
>          Issue Type: Improvement
>          Components: beam-model, sdk-java-core, sdk-py-core
>            Reporter: Brian Hulette
>            Assignee: Brian Hulette
>            Priority: Major
>          Time Spent: 12h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to