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

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

                Author: ASF GitHub Bot
            Created on: 06/Aug/20 17:28
            Start Date: 06/Aug/20 17:28
    Worklog Time Spent: 10m 
      Work Description: lostluck commented on a change in pull request #12471:
URL: https://github.com/apache/beam/pull/12471#discussion_r466572347



##########
File path: sdks/go/pkg/beam/core/runtime/graphx/schema/schema.go
##########
@@ -0,0 +1,245 @@
+// 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.
+
+// Package schema contains utility functions for relating Go types and Beam 
Schemas.
+//
+// Not all Go types can be converted to schemas. This is Go is more expressive 
than
+// Beam schemas. Just as not all Go types can be serialized, similarly,
+// not all Beam Schemas will have a conversion to Go types, until the correct
+// mechanism exists in the SDK to handle them.
+//
+// While efforts will be made to have conversions be reversable, this will not
+// be possible in all instances. Eg. Go arrays as fields will be converted to
+// Beam Arrays, but a Beam Array type will map by default to a Go slice.
+package schema
+
+import (
+       "fmt"
+       "reflect"
+       "strings"
+
+       "github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+       "github.com/apache/beam/sdks/go/pkg/beam/internal/errors"
+       pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+)
+
+// FromType returns a Beam Schema of the passed in type.
+// Returns an error if the type cannot be converted to a Schema.
+func FromType(ot reflect.Type) (*pipepb.Schema, error) {
+       t := ot // keep the original type for errors.
+       // The top level schema for a pointer to struct and the struct is the 
same.
+       if t.Kind() == reflect.Ptr {
+               t = t.Elem()
+       }
+       if t.Kind() != reflect.Struct {
+               return nil, errors.Errorf("cannot convert %v to schema. 
FromType only converts structs to schemas", ot)
+       }
+       return structToSchema(t), nil
+}
+
+func structToSchema(t reflect.Type) *pipepb.Schema {
+       fields := make([]*pipepb.Field, 0, t.NumField())
+       for i := 0; i < t.NumField(); i++ {
+               fields = append(fields, structFieldToField(t.Field(i)))
+       }
+       return &pipepb.Schema{
+               Fields: fields,
+       }
+}
+
+func structFieldToField(sf reflect.StructField) *pipepb.Field {
+       name := sf.Name
+       if tag := sf.Tag.Get("beam"); tag != "" {
+               name, _ = parseTag(tag)
+       }
+       ftype := reflectTypeToFieldType(sf.Type)
+
+       return &pipepb.Field{
+               Name: name,
+               Type: ftype,
+       }
+}
+
+func reflectTypeToFieldType(ot reflect.Type) *pipepb.FieldType {
+       var isPtr bool
+       t := ot
+       if t.Kind() == reflect.Ptr {
+               isPtr = true
+               t = t.Elem()
+       }
+       switch t.Kind() {
+       case reflect.Map:
+               kt := reflectTypeToFieldType(t.Key())
+               vt := reflectTypeToFieldType(t.Elem())
+               return &pipepb.FieldType{
+                       Nullable: isPtr,
+                       TypeInfo: &pipepb.FieldType_MapType{
+                               MapType: &pipepb.MapType{
+                                       KeyType:   kt,
+                                       ValueType: vt,
+                               },
+                       },
+               }
+       case reflect.Struct:
+               sch := structToSchema(t)
+               return &pipepb.FieldType{
+                       Nullable: isPtr,
+                       TypeInfo: &pipepb.FieldType_RowType{
+                               RowType: &pipepb.RowType{
+                                       Schema: sch,
+                               },
+                       },
+               }
+       case reflect.Slice, reflect.Array:
+               // Special handling for []byte
+               if t == reflectx.ByteSlice {
+                       return &pipepb.FieldType{
+                               Nullable: isPtr,
+                               TypeInfo: &pipepb.FieldType_AtomicType{
+                                       AtomicType: pipepb.AtomicType_BYTES,
+                               },
+                       }
+               }
+               vt := reflectTypeToFieldType(t.Elem())
+               return &pipepb.FieldType{
+                       Nullable: isPtr,
+                       TypeInfo: &pipepb.FieldType_ArrayType{
+                               ArrayType: &pipepb.ArrayType{
+                                       ElementType: vt,
+                               },
+                       },
+               }
+       case reflect.Interface, reflect.Chan, reflect.UnsafePointer, 
reflect.Complex128, reflect.Complex64, reflect.Int:
+               panic(fmt.Sprintf("Unsupported type to convert to schema: %v", 
ot))

Review comment:
       Good catch. I was definitely doing this out of iteration laziness. 
Handling error propagation now.




----------------------------------------------------------------
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:
[email protected]


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

    Worklog Id:     (was: 467440)
    Time Spent: 10h 20m  (was: 10h 10m)

> [Go SDK] Beam Schemas
> ---------------------
>
>                 Key: BEAM-9615
>                 URL: https://issues.apache.org/jira/browse/BEAM-9615
>             Project: Beam
>          Issue Type: New Feature
>          Components: sdk-go
>            Reporter: Robert Burke
>            Assignee: Robert Burke
>            Priority: P2
>          Time Spent: 10h 20m
>  Remaining Estimate: 0h
>
> Schema support is required for advanced cross language features in Beam, and 
> has the opportunity to replace the current default JSON encoding of elements.
> Some quick notes, though a better fleshed out doc with details will be 
> forthcoming:
>  * All base coders should be implemented, and listed as coder capabilities. I 
> think only stringutf8 is missing presently.
>  * Should support fairly arbitrary user types, seamlessly. That is, users 
> should be able to rely on it "just working" if their type is compatible.
>  * Should support schema metadata tagging.
> In particular, one breaking shift in the default will be to explicitly fail 
> pipelines if elements have unexported fields, when no other custom coder has 
> been added. This has been a source of errors/dropped data/keys and a simply 
> warning at construction time won't cut it. However, we could provide a manual 
> "use beam schemas, but ignore unexported fields" registration as a work 
> around.
> Edit: Doc is now at https://s.apache.org/beam-go-schemas



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

Reply via email to