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

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

                Author: ASF GitHub Bot
            Created on: 24/Aug/19 01:02
            Start Date: 24/Aug/19 01:02
    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_r317338268
 
 

 ##########
 File path: sdks/python/apache_beam/typehints/schemas.py
 ##########
 @@ -0,0 +1,217 @@
+#
+# 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.
+#
+
+""" Support for mapping python types to proto Schemas and back again.
+
+Python              Schema
+np.int8     <-----> BYTE
+np.int16    <-----> INT16
+np.int32    <-----> INT32
+np.int64    <-----> INT64
+int         ---/
+np.float32  <-----> FLOAT
+np.float64  <-----> DOUBLE
+float       ---/
+bool        <-----> BOOLEAN
+
+The mappings for STRING and BYTES are different between python 2 and python 3,
+because of the changes to str:
+py3:
+str/unicode <-----> STRING
+bytes       <-----> BYTES
+ByteString  ---/
+
+py2:
+unicode     <-----> STRING
+str/bytes   ---/
+ByteString  <-----> BYTES
+"""
+
+from __future__ import absolute_import
+
+import sys
+from typing import ByteString
+from typing import Mapping
+from typing import NamedTuple
+from typing import Optional
+from typing import Sequence
+from uuid import uuid4
+
+import numpy as np
+from past.builtins import unicode
+
+from apache_beam.portability.api import schema_pb2
+from apache_beam.typehints.native_type_compatibility import _get_args
+from apache_beam.typehints.native_type_compatibility import 
_match_is_exactly_mapping
+from apache_beam.typehints.native_type_compatibility import 
_match_is_named_tuple
+from apache_beam.typehints.native_type_compatibility import _match_is_optional
+from apache_beam.typehints.native_type_compatibility import _safe_issubclass
+from apache_beam.typehints.native_type_compatibility import 
extract_optional_type
+
+
+# Registry of typings for a schema by UUID
+class SchemaTypeRegistry(object):
+  def __init__(self):
+    self.by_id = {}
+    self.by_typing = {}
+
+  def add(self, typing, schema):
+    self.by_id[schema.id] = (typing, schema)
+
+  def get_typing_by_id(self, unique_id):
+    result = self.by_id.get(unique_id, None)
+    return result[0] if result is not None else None
+
+  def get_schema_by_id(self, unique_id):
+    result = self.by_id.get(unique_id, None)
+    return result[1] if result is not None else None
+
+
+SCHEMA_REGISTRY = SchemaTypeRegistry()
+
+
+# Bi-directional mappings
+_PRIMITIVES = (
+    (np.int8, schema_pb2.AtomicType.BYTE),
+    (np.int16, schema_pb2.AtomicType.INT16),
+    (np.int32, schema_pb2.AtomicType.INT32),
+    (np.int64, schema_pb2.AtomicType.INT64),
+    (np.float32, schema_pb2.AtomicType.FLOAT),
+    (np.float64, schema_pb2.AtomicType.DOUBLE),
+    (unicode, schema_pb2.AtomicType.STRING),
+    (bool, schema_pb2.AtomicType.BOOLEAN),
+    (bytes if sys.version_info.major >= 3 else ByteString,
+     schema_pb2.AtomicType.BYTES),
+)
+
+PRIMITIVE_TO_ATOMIC_TYPE = dict((typ, atomic) for typ, atomic in _PRIMITIVES)
+ATOMIC_TYPE_TO_PRIMITIVE = dict((atomic, typ) for typ, atomic in _PRIMITIVES)
+
+# One-way mappings
+PRIMITIVE_TO_ATOMIC_TYPE.update({
+    # In python 3, this is a no-op because str == unicode,
+    # but in python 2 it overrides the bytes -> BYTES mapping.
+    str: schema_pb2.AtomicType.STRING,
 
 Review comment:
   I think that makes sense. It doesn't seem too onerous to ask people to use 
unicode in python 2. And that's a good point that it's a backwards compatible 
change if we find out otherwise. I pushed a commit that does this: 
https://github.com/apache/beam/pull/9188/commits/ecaf73cd8eb82739c21f0a3551efa0d03cc842b6
 
----------------------------------------------------------------
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: 300639)
    Time Spent: 10h  (was: 9h 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: 10h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.2#803003)

Reply via email to