You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/08/14 21:15:37 UTC

[GitHub] [beam] lostluck opened a new pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

lostluck opened a new pull request #12588:
URL: https://github.com/apache/beam/pull/12588


   This PR has the Go SDK join Java and Python in having its coders tested against the standard_coder.yaml suite.
   
   * It adds representation markers for standard types that previously didn't have representation (such as iterables, rows, length prefixes, etc) in the graph/coder package.
   * It adds execution wrappers for encoding, decoding those values in the exec package.
   * It adds serialization support for these representations in the graphx package.
   * Adds hashing support for row coded values.
   
   Testing of the code here is largely through the newly introduced fromyaml test harness which reads standard_coder.yaml and helps it along a touch, in particular in how to interpret the yaml versions of expected structures, which don't map to go types exactly.
   
   While this PR plumbs in Timers and ParamWindowCoders, these coders are not implemented in the SDK at this time, and those tests are skipped.  At best, the implementation is vestigial, but leaving them wholely unimplemented caused certain difficulties with the standard yaml test.
   Similarly, while Windows are now fully plumbed through the stack, this is largely for convenience when testing those codings vs the standard_coders.yaml file. 
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Format the pull request title like `[BEAM-XXX] Fixes bug in ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   Post-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   Lang | SDK | Dataflow | Flink | Samza | Spark | Twister2
   --- | --- | --- | --- | --- | --- | ---
   Go | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/) | ---
   Java | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/badge/i
 con)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/)<br>[![Build Status](htt
 ps://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/)
   Python | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_
 Py_VR_Dataflow/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_P
 ostCommit_Python_VR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/) | ---
   XLang | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/) | ---
   
   Pre-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   --- |Java | Python | Go | Website
   --- | --- | --- | --- | ---
   Non-portable | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/b
 eam_PreCommit_Website_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/)
   Portable | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/) | --- | ---
   
   See [.test-infra/jenkins/README](https://github.com/apache/beam/blob/master/.test-infra/jenkins/README.md) for trigger phrase, status and link of all Jenkins jobs.
   
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   ![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] TheNeuralBit commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-693698195


   Sorry about that @lostluck, I should've let you know about the new test case. There should probably be an easier way for SDKs to temporarily opt out of particular tests. WDYT about giving each suite labels that SDKs can use in an ignorelist?


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-676525572


   Run Python PreCommit


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on a change in pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on a change in pull request #12588:
URL: https://github.com/apache/beam/pull/12588#discussion_r488226218



##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {
+		err := c.dec.DecodeTo(r, &e)
+		if err != nil {
+			return reflect.Value{}, err
+		}
+		if e.Elm != nil {
+			rv.Index(i).Set(reflect.ValueOf(e.Elm))
+		} else {
+			rv.Index(i).Set(reflect.ValueOf(e.Windows[0]))

Review comment:
       In this case it's to set it to the 0th window element. In this case it's a small hack to support "window's as element values" which is only necessary to dramatically simplify the yaml tests which actually test the coder values here. 
   
   Deserves a comment to explain it, as it is a hack.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] youngoli commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
youngoli commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-690889052






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] youngoli commented on a change in pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #12588:
URL: https://github.com/apache/beam/pull/12588#discussion_r486767704



##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -81,24 +82,82 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder {
 		return &stringEncoder{}
 
 	case coder.Custom:
-		return &customEncoder{
+		enc := &customEncoder{
 			t:   c.Custom.Type,
 			enc: makeEncoder(c.Custom.Enc.Fn),
 		}
+		if c.Custom.Name != "schema" {
+			return enc
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.

Review comment:
       I don't really understand this comment, or how it relates to the check below.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)

Review comment:
       Looks like another debug println.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -534,26 +1038,34 @@ func (*intervalWindowEncoder) EncodeSingle(elm typex.Window, w io.Writer) error
 
 type intervalWindowDecoder struct{}
 
-func (*intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
+func (d *intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
 	// Encoding: upper bound and duration
 
 	n, err := coder.DecodeInt32(r) // #windows
 
 	ret := make([]typex.Window, n, n)
 	for i := int32(0); i < n; i++ {
-		end, err := coder.DecodeEventTime(r)
-		if err != nil {
-			return nil, err
-		}
-		duration, err := coder.DecodeVarUint64(r)
+		w, err := d.DecodeSingle(r)
 		if err != nil {
 			return nil, err
 		}
-		ret[i] = window.IntervalWindow{Start: mtime.FromMilliseconds(end.Milliseconds() - int64(duration)), End: end}
+		ret[i] = w
 	}
 	return ret, err
 }
 
+func (*intervalWindowDecoder) DecodeSingle(r io.Reader) (typex.Window, error) {
+	end, err := coder.DecodeEventTime(r)
+	if err != nil {
+		return nil, err
+	}
+	duration, err := coder.DecodeVarInt(r)

Review comment:
       What's the reason to switch from `DecodeVarUint64` to `DecodeVarInt` here?

##########
File path: sdks/go/test/regression/coders/fromyaml/fromyaml.go
##########
@@ -0,0 +1,415 @@
+// 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.
+
+// fromyaml generates a resource file from the standard_coders.yaml
+// file for use in these coder regression tests.
+//
+// It expects to be run in it's test directory, or via it's go test.
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"math"
+	"reflect"
+	"runtime/debug"
+	"strconv"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/google/go-cmp/cmp"
+	"golang.org/x/text/encoding/charmap"
+	yaml "gopkg.in/yaml.v2"
+)
+
+var unimplementedCoders = map[string]bool{
+	"beam:coder:param_windowed_value:v1": true,
+	"beam:coder:timer:v1":                true,
+}
+
+// Coder is a representation a serialized beam coder.
+type Coder struct {
+	Urn              string  `yaml:"urn,omitempty"`
+	Payload          string  `yaml:"payload,omitempty"`
+	Components       []Coder `yaml:"components,omitempty"`
+	NonDeterministic bool    `yaml:"non_deterministic,omitempty"`
+}
+
+type logger interface {
+	Errorf(string, ...interface{})
+	Logf(string, ...interface{})
+}
+
+// Spec is a set of conditions that a coder must pass.
+type Spec struct {
+	Coder    Coder         `yaml:"coder,omitempty"`
+	Nested   *bool         `yaml:"nested,omitempty"`
+	Examples yaml.MapSlice `yaml:"examples,omitempty"`
+	Log      logger
+
+	id       int // for generating coder ids.
+	coderPBs map[string]*pipepb.Coder
+}
+
+func (s *Spec) nextID() string {
+	ret := fmt.Sprintf("%d", s.id)
+	s.id++
+	return ret
+}
+
+func (s *Spec) testStandardCoder() (err error) {
+	if unimplementedCoders[s.Coder.Urn] {
+		log.Printf("skipping unimplemented coder urn: %v", s.Coder.Urn)
+		return nil
+	}
+	// Construct the coder proto equivalents.
+
+	// Only nested tests need to be run, since nestedness is a pre-portability
+	// concept.
+	// For legacy Java reasons, the row coder examples are all marked nested: false
+	// so we need to check that before skipping unnested tests.
+	if s.Coder.Urn != "beam:coder:row:v1" && s.Nested != nil && !*s.Nested {
+		log.Printf("skipping unnested coder spec: %v\n", s.Coder)
+		return nil
+	}
+
+	s.coderPBs = make(map[string]*pipepb.Coder)
+	id := s.parseCoder(s.Coder)
+	b := graphx.NewCoderUnmarshaller(s.coderPBs)
+	underTest, err := b.Coder(id)
+	if err != nil {
+		return fmt.Errorf("unable to create coder: %v", err)
+	}
+
+	defer func() {
+		if e := recover(); e != nil {
+			err = fmt.Errorf("panicked on coder %v || %v:\n\t%v :\n%s", underTest, s.Coder, e, debug.Stack())
+		}
+	}()
+
+	var decFails, encFails int
+	for _, eg := range s.Examples {
+
+		// Test Decoding
+		// Ideally we'd use the beam package coders, but KVs make that complicated.
+		// This can be cleaned up once a type parametered beam.KV type exists.
+		dec := exec.MakeElementDecoder(underTest)
+		encoded := eg.Key.(string)
+		var elem exec.FullValue
+
+		// What I would have expected.
+		//		r := charmap.ISO8859_1.NewDecoder().Reader(strings.NewReader(encoded))
+		recoded, err := charmap.ISO8859_1.NewEncoder().String(encoded)
+		if err != nil {
+			return err
+		}
+		r := strings.NewReader(recoded)
+		if err := dec.DecodeTo(r, &elem); err != nil {
+			return fmt.Errorf("err decoding %q: %v", encoded, err)
+		}
+		if !diff(s.Coder, &elem, eg) {
+			decFails++
+			continue
+		}
+
+		// Test Encoding
+		if s.Coder.NonDeterministic {
+			// Skip verifying nondeterministic encodings.
+			continue
+		}
+		enc := exec.MakeElementEncoder(underTest)
+		var out bytes.Buffer
+		if err := enc.Encode(&elem, &out); err != nil {
+			return err
+		}
+		if d := cmp.Diff(recoded, string(out.Bytes())); d != "" {
+			log.Printf("Encoding error: diff(-want,+got): %v\n", d)
+		}
+	}
+	if decFails+encFails > 0 {
+		return fmt.Errorf("failed to decode %v times, and encode %v times", decFails, encFails)
+	}
+
+	return nil
+}
+
+var cmpOpts = []cmp.Option{
+	cmp.Transformer("bytes2string", func(in []byte) (out string) {
+		return string(in)
+	}),
+}
+
+func diff(c Coder, elem *exec.FullValue, eg yaml.MapItem) bool {
+	var got, want interface{}
+	switch c.Urn {
+	case "beam:coder:bytes:v1":
+		got = string(elem.Elm.([]byte))
+		switch egv := eg.Value.(type) {
+		case string:
+			want = egv
+		case []byte:
+			want = string(egv)
+		}
+	case "beam:coder:varint:v1":
+		got, want = elem.Elm.(int64), int64(eg.Value.(int))
+	case "beam:coder:double:v1":
+		got = elem.Elm.(float64)
+		switch v := eg.Value.(string); v {
+		case "NaN":
+			// Do the NaN comparison here since NaN by definition != NaN.
+			if math.IsNaN(got.(float64)) {
+				want, got = 1, 1
+			} else {
+				want = math.NaN()
+			}
+		case "-Infinity":
+			want = math.Inf(-1)
+		case "Infinity":
+			want = math.Inf(1)
+		default:
+			want, _ = strconv.ParseFloat(v, 64)
+		}
+
+	case "beam:coder:kv:v1":
+		v := eg.Value.(yaml.MapSlice)
+		pass := true
+		if !diff(c.Components[0], &exec.FullValue{Elm: elem.Elm}, v[0]) {
+			pass = false
+		}
+		if !diff(c.Components[1], &exec.FullValue{Elm: elem.Elm2}, v[1]) {
+			pass = false
+		}
+		return pass
+
+	case "beam:coder:iterable:v1":
+		pass := true
+		gotrv := reflect.ValueOf(elem.Elm)
+		wantrv := reflect.ValueOf(eg.Value)
+		if gotrv.Len() != wantrv.Len() {
+			log.Printf("Lengths don't match. got %v, want %v;  %v, %v", gotrv.Len(), wantrv.Len(), gotrv, wantrv)
+			return false
+		}
+		for i := 0; i < wantrv.Len(); i++ {
+			if !diff(c.Components[0],
+				&exec.FullValue{Elm: gotrv.Index(i).Interface()},
+				yaml.MapItem{Value: wantrv.Index(i).Interface()}) {
+				pass = false
+			}
+
+		}
+		return pass
+	case "beam:coder:interval_window:v1":
+		var a, b int
+		val := eg.Value
+		if is, ok := eg.Value.([]interface{}); ok {
+			val = is[0]
+		}
+		v := val.(yaml.MapSlice)
+
+		a = v[0].Value.(int)
+		b = v[1].Value.(int)
+		end := mtime.FromMilliseconds(int64(a))
+		start := end - mtime.Time(int64(b))
+		want = window.IntervalWindow{Start: start, End: end}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = elem.Elm
+		} else {
+			got = elem.Windows[0]
+		}
+
+	case "beam:coder:global_window:v1":
+		want = window.GlobalWindow{}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = window.GlobalWindow(elem.Elm.(struct{}))
+		} else {
+			got = elem.Windows[0]
+		}
+	case "beam:coder:windowed_value:v1", "beam:coder:param_windowed_value:v1":
+		// elem contains all the information, but we need to compare the element+timestamp
+		// separately from the windows, to avoid repeated expected value parsing logic.
+		pass := true
+		vs := eg.Value.(yaml.MapSlice)
+		if !diff(c.Components[0], elem, vs[0]) {
+			pass = false
+		}
+		if d := cmp.Diff(
+			mtime.FromMilliseconds(int64(vs[1].Value.(int))),
+			elem.Timestamp, cmpOpts...); d != "" {
+
+			pass = false
+		}
+		if !diff(c.Components[1], elem, vs[3]) {
+			pass = false
+		}
+		// TODO compare pane information.
+		return pass
+	case "beam:coder:row:v1":
+		fs := eg.Value.(yaml.MapSlice)
+		var rfs []reflect.StructField
+		// There are only 2 pointer examples, but they reuse field names,
+		// so we key off the proto hash to know which example we're handling.
+		ptrEg := strings.Contains(c.Payload, "51ace21c7393")
+		for _, rf := range fs {
+			name := rf.Key.(string)
+			t := nameToType[name]
+			if ptrEg {
+				t = reflect.PtrTo(t)
+			}
+			rfs = append(rfs, reflect.StructField{
+				Name: strings.ToUpper(name[:1]) + name[1:],
+				Type: t,
+				Tag:  reflect.StructTag(fmt.Sprintf("beam:\"%v\"", name)),
+			})
+		}
+		rv := reflect.New(reflect.StructOf(rfs)).Elem()
+		for i, rf := range fs {
+			setField(rv, i, rf.Value)
+		}
+
+		got, want = elem.Elm, rv.Interface()
+	default:
+		got, want = elem.Elm, eg.Value
+	}
+	if d := cmp.Diff(want, got, cmpOpts...); d != "" {
+		log.Printf("Decoding error: diff(-want,+got): %v\n", d)
+		return false
+	}
+	return true
+}
+
+// standard_coders.yaml uses the name for type indication, except for nullability.
+var nameToType = map[string]reflect.Type{
+	"str":     reflectx.String,
+	"i32":     reflectx.Int32,
+	"f64":     reflectx.Float64,
+	"arr":     reflect.SliceOf(reflectx.String),
+	"f_bool":  reflectx.Bool,
+	"f_bytes": reflect.PtrTo(reflectx.ByteSlice),
+	"f_map":   reflect.MapOf(reflectx.String, reflect.PtrTo(reflectx.Int64)),
+}
+
+func setField(rv reflect.Value, i int, v interface{}) {
+	if v == nil {
+		return
+	}
+	rf := rv.Field(i)
+	if rf.Kind() == reflect.Ptr {
+		// Ensure it's initialized.
+		rf.Set(reflect.New(rf.Type().Elem()))
+		rf = rf.Elem()
+	}
+	switch rf.Kind() {
+	case reflect.String:
+		rf.SetString(v.(string))
+	case reflect.Int32:
+		rf.SetInt(int64(v.(int)))
+	case reflect.Float64:
+		c, err := strconv.ParseFloat(v.(string), 64)
+		if err != nil {
+			panic(err)
+		}
+		rf.SetFloat(c)
+	case reflect.Slice:
+		if rf.Type() == reflectx.ByteSlice {
+			rf.Set(reflect.ValueOf([]byte(v.(string))))
+			break
+		}
+		// Value is a []interface{} with string values.

Review comment:
       Is this assumption made specifically based on the yaml file this is meant to be used with? Or is this based on something in the schema spec I'm forgetting?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)
+		if c.Custom.Name != "schema" {
+			return dec
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.
+		switch c.T.Type().Kind() {
+		case reflect.Slice:
+			return &lpDecoder{
+				dec: &iterableDecoder{
+					t:   c.Custom.Type,
+					dec: dec,
+				},
+			}
+		case reflect.Array:
+			return &lpDecoder{
+				dec: &arrayDecoder{

Review comment:
       Just to confirm, using an `arrayDecoder` without an `arrayEncoder` specifically works because the format that `iterableEncoder` encodes to is also decodable as an array, right? That might be worth mentioning as a comment, because I was wondering why you can decode arrays with an `arrayDecoder` if they were encoded with an `iterableEncoder`.

##########
File path: sdks/go/pkg/beam/core/runtime/graphx/coder.go
##########
@@ -249,31 +256,35 @@ func (b *CoderUnmarshaller) makeCoder(c *pipepb.Coder) (*coder.Coder, error) {
 			return nil, err
 		}
 
-		// No payload means this coder was length prefixed by the runner
-		// but is likely self describing - AKA a beam coder.
-		if len(sub.GetSpec().GetPayload()) == 0 {
-			return b.makeCoder(sub)
-		}
 		// TODO(lostluck) 2018/10/17: Make this strict again, once dataflow can use
 		// the portable pipeline model directly (BEAM-2885)
-		if sub.GetSpec().GetUrn() != "" && sub.GetSpec().GetUrn() != urnCustomCoder {
-			// TODO(herohde) 11/17/2017: revisit this restriction
-			return nil, errors.Errorf("could not unmarshal length prefix coder from %v, want a custom coder as a sub component but got %v", c, sub)
-		}
-
-		var ref v1pb.CustomCoder
-		if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
-			return nil, err
-		}
-		custom, err := decodeCustomCoder(&ref)
-		if err != nil {
-			return nil, err
+		switch u := sub.GetSpec().GetUrn(); u {
+		case "", urnCustomCoder:
+			var ref v1pb.CustomCoder
+			if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
+				return nil, err
+			}
+			custom, err := decodeCustomCoder(&ref)
+			if err != nil {
+				return nil, err
+			}
+			custom.ID = components[0]
+			t := typex.New(custom.Type)
+			cc := &coder.Coder{Kind: coder.Custom, T: t, Custom: custom}
+			fmt.Println("decoded customcoder", cc)

Review comment:
       Looks like a debug Println left in.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)

Review comment:
       Is it necessary to cast n to an int here? Isn't it already an int from line 638?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {
+		err := c.dec.DecodeTo(r, &e)
+		if err != nil {
+			return reflect.Value{}, err
+		}
+		if e.Elm != nil {
+			rv.Index(i).Set(reflect.ValueOf(e.Elm))
+		} else {
+			rv.Index(i).Set(reflect.ValueOf(e.Windows[0]))

Review comment:
       I don't get why it's setting the value of the element to `e.Windows[0]`. Is it a way to pass in nil?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {

Review comment:
       Likewise, is it necessary to cast n here too?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -81,24 +82,82 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder {
 		return &stringEncoder{}
 
 	case coder.Custom:
-		return &customEncoder{
+		enc := &customEncoder{
 			t:   c.Custom.Type,
 			enc: makeEncoder(c.Custom.Enc.Fn),
 		}
+		if c.Custom.Name != "schema" {
+			return enc
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.

Review comment:
       I don't really understand this comment, or how it relates to the check below.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)

Review comment:
       Looks like another debug println.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -534,26 +1038,34 @@ func (*intervalWindowEncoder) EncodeSingle(elm typex.Window, w io.Writer) error
 
 type intervalWindowDecoder struct{}
 
-func (*intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
+func (d *intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
 	// Encoding: upper bound and duration
 
 	n, err := coder.DecodeInt32(r) // #windows
 
 	ret := make([]typex.Window, n, n)
 	for i := int32(0); i < n; i++ {
-		end, err := coder.DecodeEventTime(r)
-		if err != nil {
-			return nil, err
-		}
-		duration, err := coder.DecodeVarUint64(r)
+		w, err := d.DecodeSingle(r)
 		if err != nil {
 			return nil, err
 		}
-		ret[i] = window.IntervalWindow{Start: mtime.FromMilliseconds(end.Milliseconds() - int64(duration)), End: end}
+		ret[i] = w
 	}
 	return ret, err
 }
 
+func (*intervalWindowDecoder) DecodeSingle(r io.Reader) (typex.Window, error) {
+	end, err := coder.DecodeEventTime(r)
+	if err != nil {
+		return nil, err
+	}
+	duration, err := coder.DecodeVarInt(r)

Review comment:
       What's the reason to switch from `DecodeVarUint64` to `DecodeVarInt` here?

##########
File path: sdks/go/test/regression/coders/fromyaml/fromyaml.go
##########
@@ -0,0 +1,415 @@
+// 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.
+
+// fromyaml generates a resource file from the standard_coders.yaml
+// file for use in these coder regression tests.
+//
+// It expects to be run in it's test directory, or via it's go test.
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"math"
+	"reflect"
+	"runtime/debug"
+	"strconv"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/google/go-cmp/cmp"
+	"golang.org/x/text/encoding/charmap"
+	yaml "gopkg.in/yaml.v2"
+)
+
+var unimplementedCoders = map[string]bool{
+	"beam:coder:param_windowed_value:v1": true,
+	"beam:coder:timer:v1":                true,
+}
+
+// Coder is a representation a serialized beam coder.
+type Coder struct {
+	Urn              string  `yaml:"urn,omitempty"`
+	Payload          string  `yaml:"payload,omitempty"`
+	Components       []Coder `yaml:"components,omitempty"`
+	NonDeterministic bool    `yaml:"non_deterministic,omitempty"`
+}
+
+type logger interface {
+	Errorf(string, ...interface{})
+	Logf(string, ...interface{})
+}
+
+// Spec is a set of conditions that a coder must pass.
+type Spec struct {
+	Coder    Coder         `yaml:"coder,omitempty"`
+	Nested   *bool         `yaml:"nested,omitempty"`
+	Examples yaml.MapSlice `yaml:"examples,omitempty"`
+	Log      logger
+
+	id       int // for generating coder ids.
+	coderPBs map[string]*pipepb.Coder
+}
+
+func (s *Spec) nextID() string {
+	ret := fmt.Sprintf("%d", s.id)
+	s.id++
+	return ret
+}
+
+func (s *Spec) testStandardCoder() (err error) {
+	if unimplementedCoders[s.Coder.Urn] {
+		log.Printf("skipping unimplemented coder urn: %v", s.Coder.Urn)
+		return nil
+	}
+	// Construct the coder proto equivalents.
+
+	// Only nested tests need to be run, since nestedness is a pre-portability
+	// concept.
+	// For legacy Java reasons, the row coder examples are all marked nested: false
+	// so we need to check that before skipping unnested tests.
+	if s.Coder.Urn != "beam:coder:row:v1" && s.Nested != nil && !*s.Nested {
+		log.Printf("skipping unnested coder spec: %v\n", s.Coder)
+		return nil
+	}
+
+	s.coderPBs = make(map[string]*pipepb.Coder)
+	id := s.parseCoder(s.Coder)
+	b := graphx.NewCoderUnmarshaller(s.coderPBs)
+	underTest, err := b.Coder(id)
+	if err != nil {
+		return fmt.Errorf("unable to create coder: %v", err)
+	}
+
+	defer func() {
+		if e := recover(); e != nil {
+			err = fmt.Errorf("panicked on coder %v || %v:\n\t%v :\n%s", underTest, s.Coder, e, debug.Stack())
+		}
+	}()
+
+	var decFails, encFails int
+	for _, eg := range s.Examples {
+
+		// Test Decoding
+		// Ideally we'd use the beam package coders, but KVs make that complicated.
+		// This can be cleaned up once a type parametered beam.KV type exists.
+		dec := exec.MakeElementDecoder(underTest)
+		encoded := eg.Key.(string)
+		var elem exec.FullValue
+
+		// What I would have expected.
+		//		r := charmap.ISO8859_1.NewDecoder().Reader(strings.NewReader(encoded))
+		recoded, err := charmap.ISO8859_1.NewEncoder().String(encoded)
+		if err != nil {
+			return err
+		}
+		r := strings.NewReader(recoded)
+		if err := dec.DecodeTo(r, &elem); err != nil {
+			return fmt.Errorf("err decoding %q: %v", encoded, err)
+		}
+		if !diff(s.Coder, &elem, eg) {
+			decFails++
+			continue
+		}
+
+		// Test Encoding
+		if s.Coder.NonDeterministic {
+			// Skip verifying nondeterministic encodings.
+			continue
+		}
+		enc := exec.MakeElementEncoder(underTest)
+		var out bytes.Buffer
+		if err := enc.Encode(&elem, &out); err != nil {
+			return err
+		}
+		if d := cmp.Diff(recoded, string(out.Bytes())); d != "" {
+			log.Printf("Encoding error: diff(-want,+got): %v\n", d)
+		}
+	}
+	if decFails+encFails > 0 {
+		return fmt.Errorf("failed to decode %v times, and encode %v times", decFails, encFails)
+	}
+
+	return nil
+}
+
+var cmpOpts = []cmp.Option{
+	cmp.Transformer("bytes2string", func(in []byte) (out string) {
+		return string(in)
+	}),
+}
+
+func diff(c Coder, elem *exec.FullValue, eg yaml.MapItem) bool {
+	var got, want interface{}
+	switch c.Urn {
+	case "beam:coder:bytes:v1":
+		got = string(elem.Elm.([]byte))
+		switch egv := eg.Value.(type) {
+		case string:
+			want = egv
+		case []byte:
+			want = string(egv)
+		}
+	case "beam:coder:varint:v1":
+		got, want = elem.Elm.(int64), int64(eg.Value.(int))
+	case "beam:coder:double:v1":
+		got = elem.Elm.(float64)
+		switch v := eg.Value.(string); v {
+		case "NaN":
+			// Do the NaN comparison here since NaN by definition != NaN.
+			if math.IsNaN(got.(float64)) {
+				want, got = 1, 1
+			} else {
+				want = math.NaN()
+			}
+		case "-Infinity":
+			want = math.Inf(-1)
+		case "Infinity":
+			want = math.Inf(1)
+		default:
+			want, _ = strconv.ParseFloat(v, 64)
+		}
+
+	case "beam:coder:kv:v1":
+		v := eg.Value.(yaml.MapSlice)
+		pass := true
+		if !diff(c.Components[0], &exec.FullValue{Elm: elem.Elm}, v[0]) {
+			pass = false
+		}
+		if !diff(c.Components[1], &exec.FullValue{Elm: elem.Elm2}, v[1]) {
+			pass = false
+		}
+		return pass
+
+	case "beam:coder:iterable:v1":
+		pass := true
+		gotrv := reflect.ValueOf(elem.Elm)
+		wantrv := reflect.ValueOf(eg.Value)
+		if gotrv.Len() != wantrv.Len() {
+			log.Printf("Lengths don't match. got %v, want %v;  %v, %v", gotrv.Len(), wantrv.Len(), gotrv, wantrv)
+			return false
+		}
+		for i := 0; i < wantrv.Len(); i++ {
+			if !diff(c.Components[0],
+				&exec.FullValue{Elm: gotrv.Index(i).Interface()},
+				yaml.MapItem{Value: wantrv.Index(i).Interface()}) {
+				pass = false
+			}
+
+		}
+		return pass
+	case "beam:coder:interval_window:v1":
+		var a, b int
+		val := eg.Value
+		if is, ok := eg.Value.([]interface{}); ok {
+			val = is[0]
+		}
+		v := val.(yaml.MapSlice)
+
+		a = v[0].Value.(int)
+		b = v[1].Value.(int)
+		end := mtime.FromMilliseconds(int64(a))
+		start := end - mtime.Time(int64(b))
+		want = window.IntervalWindow{Start: start, End: end}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = elem.Elm
+		} else {
+			got = elem.Windows[0]
+		}
+
+	case "beam:coder:global_window:v1":
+		want = window.GlobalWindow{}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = window.GlobalWindow(elem.Elm.(struct{}))
+		} else {
+			got = elem.Windows[0]
+		}
+	case "beam:coder:windowed_value:v1", "beam:coder:param_windowed_value:v1":
+		// elem contains all the information, but we need to compare the element+timestamp
+		// separately from the windows, to avoid repeated expected value parsing logic.
+		pass := true
+		vs := eg.Value.(yaml.MapSlice)
+		if !diff(c.Components[0], elem, vs[0]) {
+			pass = false
+		}
+		if d := cmp.Diff(
+			mtime.FromMilliseconds(int64(vs[1].Value.(int))),
+			elem.Timestamp, cmpOpts...); d != "" {
+
+			pass = false
+		}
+		if !diff(c.Components[1], elem, vs[3]) {
+			pass = false
+		}
+		// TODO compare pane information.
+		return pass
+	case "beam:coder:row:v1":
+		fs := eg.Value.(yaml.MapSlice)
+		var rfs []reflect.StructField
+		// There are only 2 pointer examples, but they reuse field names,
+		// so we key off the proto hash to know which example we're handling.
+		ptrEg := strings.Contains(c.Payload, "51ace21c7393")
+		for _, rf := range fs {
+			name := rf.Key.(string)
+			t := nameToType[name]
+			if ptrEg {
+				t = reflect.PtrTo(t)
+			}
+			rfs = append(rfs, reflect.StructField{
+				Name: strings.ToUpper(name[:1]) + name[1:],
+				Type: t,
+				Tag:  reflect.StructTag(fmt.Sprintf("beam:\"%v\"", name)),
+			})
+		}
+		rv := reflect.New(reflect.StructOf(rfs)).Elem()
+		for i, rf := range fs {
+			setField(rv, i, rf.Value)
+		}
+
+		got, want = elem.Elm, rv.Interface()
+	default:
+		got, want = elem.Elm, eg.Value
+	}
+	if d := cmp.Diff(want, got, cmpOpts...); d != "" {
+		log.Printf("Decoding error: diff(-want,+got): %v\n", d)
+		return false
+	}
+	return true
+}
+
+// standard_coders.yaml uses the name for type indication, except for nullability.
+var nameToType = map[string]reflect.Type{
+	"str":     reflectx.String,
+	"i32":     reflectx.Int32,
+	"f64":     reflectx.Float64,
+	"arr":     reflect.SliceOf(reflectx.String),
+	"f_bool":  reflectx.Bool,
+	"f_bytes": reflect.PtrTo(reflectx.ByteSlice),
+	"f_map":   reflect.MapOf(reflectx.String, reflect.PtrTo(reflectx.Int64)),
+}
+
+func setField(rv reflect.Value, i int, v interface{}) {
+	if v == nil {
+		return
+	}
+	rf := rv.Field(i)
+	if rf.Kind() == reflect.Ptr {
+		// Ensure it's initialized.
+		rf.Set(reflect.New(rf.Type().Elem()))
+		rf = rf.Elem()
+	}
+	switch rf.Kind() {
+	case reflect.String:
+		rf.SetString(v.(string))
+	case reflect.Int32:
+		rf.SetInt(int64(v.(int)))
+	case reflect.Float64:
+		c, err := strconv.ParseFloat(v.(string), 64)
+		if err != nil {
+			panic(err)
+		}
+		rf.SetFloat(c)
+	case reflect.Slice:
+		if rf.Type() == reflectx.ByteSlice {
+			rf.Set(reflect.ValueOf([]byte(v.(string))))
+			break
+		}
+		// Value is a []interface{} with string values.

Review comment:
       Is this assumption made specifically based on the yaml file this is meant to be used with? Or is this based on something in the schema spec I'm forgetting?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)
+		if c.Custom.Name != "schema" {
+			return dec
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.
+		switch c.T.Type().Kind() {
+		case reflect.Slice:
+			return &lpDecoder{
+				dec: &iterableDecoder{
+					t:   c.Custom.Type,
+					dec: dec,
+				},
+			}
+		case reflect.Array:
+			return &lpDecoder{
+				dec: &arrayDecoder{

Review comment:
       Just to confirm, using an `arrayDecoder` without an `arrayEncoder` specifically works because the format that `iterableEncoder` encodes to is also decodable as an array, right? That might be worth mentioning as a comment, because I was wondering why you can decode arrays with an `arrayDecoder` if they were encoded with an `iterableEncoder`.

##########
File path: sdks/go/pkg/beam/core/runtime/graphx/coder.go
##########
@@ -249,31 +256,35 @@ func (b *CoderUnmarshaller) makeCoder(c *pipepb.Coder) (*coder.Coder, error) {
 			return nil, err
 		}
 
-		// No payload means this coder was length prefixed by the runner
-		// but is likely self describing - AKA a beam coder.
-		if len(sub.GetSpec().GetPayload()) == 0 {
-			return b.makeCoder(sub)
-		}
 		// TODO(lostluck) 2018/10/17: Make this strict again, once dataflow can use
 		// the portable pipeline model directly (BEAM-2885)
-		if sub.GetSpec().GetUrn() != "" && sub.GetSpec().GetUrn() != urnCustomCoder {
-			// TODO(herohde) 11/17/2017: revisit this restriction
-			return nil, errors.Errorf("could not unmarshal length prefix coder from %v, want a custom coder as a sub component but got %v", c, sub)
-		}
-
-		var ref v1pb.CustomCoder
-		if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
-			return nil, err
-		}
-		custom, err := decodeCustomCoder(&ref)
-		if err != nil {
-			return nil, err
+		switch u := sub.GetSpec().GetUrn(); u {
+		case "", urnCustomCoder:
+			var ref v1pb.CustomCoder
+			if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
+				return nil, err
+			}
+			custom, err := decodeCustomCoder(&ref)
+			if err != nil {
+				return nil, err
+			}
+			custom.ID = components[0]
+			t := typex.New(custom.Type)
+			cc := &coder.Coder{Kind: coder.Custom, T: t, Custom: custom}
+			fmt.Println("decoded customcoder", cc)

Review comment:
       Looks like a debug Println left in.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)

Review comment:
       Is it necessary to cast n to an int here? Isn't it already an int from line 638?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {
+		err := c.dec.DecodeTo(r, &e)
+		if err != nil {
+			return reflect.Value{}, err
+		}
+		if e.Elm != nil {
+			rv.Index(i).Set(reflect.ValueOf(e.Elm))
+		} else {
+			rv.Index(i).Set(reflect.ValueOf(e.Windows[0]))

Review comment:
       I don't get why it's setting the value of the element to `e.Windows[0]`. Is it a way to pass in nil?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {

Review comment:
       Likewise, is it necessary to cast n here too?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -81,24 +82,82 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder {
 		return &stringEncoder{}
 
 	case coder.Custom:
-		return &customEncoder{
+		enc := &customEncoder{
 			t:   c.Custom.Type,
 			enc: makeEncoder(c.Custom.Enc.Fn),
 		}
+		if c.Custom.Name != "schema" {
+			return enc
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.

Review comment:
       I don't really understand this comment, or how it relates to the check below.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)

Review comment:
       Looks like another debug println.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -534,26 +1038,34 @@ func (*intervalWindowEncoder) EncodeSingle(elm typex.Window, w io.Writer) error
 
 type intervalWindowDecoder struct{}
 
-func (*intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
+func (d *intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
 	// Encoding: upper bound and duration
 
 	n, err := coder.DecodeInt32(r) // #windows
 
 	ret := make([]typex.Window, n, n)
 	for i := int32(0); i < n; i++ {
-		end, err := coder.DecodeEventTime(r)
-		if err != nil {
-			return nil, err
-		}
-		duration, err := coder.DecodeVarUint64(r)
+		w, err := d.DecodeSingle(r)
 		if err != nil {
 			return nil, err
 		}
-		ret[i] = window.IntervalWindow{Start: mtime.FromMilliseconds(end.Milliseconds() - int64(duration)), End: end}
+		ret[i] = w
 	}
 	return ret, err
 }
 
+func (*intervalWindowDecoder) DecodeSingle(r io.Reader) (typex.Window, error) {
+	end, err := coder.DecodeEventTime(r)
+	if err != nil {
+		return nil, err
+	}
+	duration, err := coder.DecodeVarInt(r)

Review comment:
       What's the reason to switch from `DecodeVarUint64` to `DecodeVarInt` here?

##########
File path: sdks/go/test/regression/coders/fromyaml/fromyaml.go
##########
@@ -0,0 +1,415 @@
+// 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.
+
+// fromyaml generates a resource file from the standard_coders.yaml
+// file for use in these coder regression tests.
+//
+// It expects to be run in it's test directory, or via it's go test.
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"math"
+	"reflect"
+	"runtime/debug"
+	"strconv"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/google/go-cmp/cmp"
+	"golang.org/x/text/encoding/charmap"
+	yaml "gopkg.in/yaml.v2"
+)
+
+var unimplementedCoders = map[string]bool{
+	"beam:coder:param_windowed_value:v1": true,
+	"beam:coder:timer:v1":                true,
+}
+
+// Coder is a representation a serialized beam coder.
+type Coder struct {
+	Urn              string  `yaml:"urn,omitempty"`
+	Payload          string  `yaml:"payload,omitempty"`
+	Components       []Coder `yaml:"components,omitempty"`
+	NonDeterministic bool    `yaml:"non_deterministic,omitempty"`
+}
+
+type logger interface {
+	Errorf(string, ...interface{})
+	Logf(string, ...interface{})
+}
+
+// Spec is a set of conditions that a coder must pass.
+type Spec struct {
+	Coder    Coder         `yaml:"coder,omitempty"`
+	Nested   *bool         `yaml:"nested,omitempty"`
+	Examples yaml.MapSlice `yaml:"examples,omitempty"`
+	Log      logger
+
+	id       int // for generating coder ids.
+	coderPBs map[string]*pipepb.Coder
+}
+
+func (s *Spec) nextID() string {
+	ret := fmt.Sprintf("%d", s.id)
+	s.id++
+	return ret
+}
+
+func (s *Spec) testStandardCoder() (err error) {
+	if unimplementedCoders[s.Coder.Urn] {
+		log.Printf("skipping unimplemented coder urn: %v", s.Coder.Urn)
+		return nil
+	}
+	// Construct the coder proto equivalents.
+
+	// Only nested tests need to be run, since nestedness is a pre-portability
+	// concept.
+	// For legacy Java reasons, the row coder examples are all marked nested: false
+	// so we need to check that before skipping unnested tests.
+	if s.Coder.Urn != "beam:coder:row:v1" && s.Nested != nil && !*s.Nested {
+		log.Printf("skipping unnested coder spec: %v\n", s.Coder)
+		return nil
+	}
+
+	s.coderPBs = make(map[string]*pipepb.Coder)
+	id := s.parseCoder(s.Coder)
+	b := graphx.NewCoderUnmarshaller(s.coderPBs)
+	underTest, err := b.Coder(id)
+	if err != nil {
+		return fmt.Errorf("unable to create coder: %v", err)
+	}
+
+	defer func() {
+		if e := recover(); e != nil {
+			err = fmt.Errorf("panicked on coder %v || %v:\n\t%v :\n%s", underTest, s.Coder, e, debug.Stack())
+		}
+	}()
+
+	var decFails, encFails int
+	for _, eg := range s.Examples {
+
+		// Test Decoding
+		// Ideally we'd use the beam package coders, but KVs make that complicated.
+		// This can be cleaned up once a type parametered beam.KV type exists.
+		dec := exec.MakeElementDecoder(underTest)
+		encoded := eg.Key.(string)
+		var elem exec.FullValue
+
+		// What I would have expected.
+		//		r := charmap.ISO8859_1.NewDecoder().Reader(strings.NewReader(encoded))
+		recoded, err := charmap.ISO8859_1.NewEncoder().String(encoded)
+		if err != nil {
+			return err
+		}
+		r := strings.NewReader(recoded)
+		if err := dec.DecodeTo(r, &elem); err != nil {
+			return fmt.Errorf("err decoding %q: %v", encoded, err)
+		}
+		if !diff(s.Coder, &elem, eg) {
+			decFails++
+			continue
+		}
+
+		// Test Encoding
+		if s.Coder.NonDeterministic {
+			// Skip verifying nondeterministic encodings.
+			continue
+		}
+		enc := exec.MakeElementEncoder(underTest)
+		var out bytes.Buffer
+		if err := enc.Encode(&elem, &out); err != nil {
+			return err
+		}
+		if d := cmp.Diff(recoded, string(out.Bytes())); d != "" {
+			log.Printf("Encoding error: diff(-want,+got): %v\n", d)
+		}
+	}
+	if decFails+encFails > 0 {
+		return fmt.Errorf("failed to decode %v times, and encode %v times", decFails, encFails)
+	}
+
+	return nil
+}
+
+var cmpOpts = []cmp.Option{
+	cmp.Transformer("bytes2string", func(in []byte) (out string) {
+		return string(in)
+	}),
+}
+
+func diff(c Coder, elem *exec.FullValue, eg yaml.MapItem) bool {
+	var got, want interface{}
+	switch c.Urn {
+	case "beam:coder:bytes:v1":
+		got = string(elem.Elm.([]byte))
+		switch egv := eg.Value.(type) {
+		case string:
+			want = egv
+		case []byte:
+			want = string(egv)
+		}
+	case "beam:coder:varint:v1":
+		got, want = elem.Elm.(int64), int64(eg.Value.(int))
+	case "beam:coder:double:v1":
+		got = elem.Elm.(float64)
+		switch v := eg.Value.(string); v {
+		case "NaN":
+			// Do the NaN comparison here since NaN by definition != NaN.
+			if math.IsNaN(got.(float64)) {
+				want, got = 1, 1
+			} else {
+				want = math.NaN()
+			}
+		case "-Infinity":
+			want = math.Inf(-1)
+		case "Infinity":
+			want = math.Inf(1)
+		default:
+			want, _ = strconv.ParseFloat(v, 64)
+		}
+
+	case "beam:coder:kv:v1":
+		v := eg.Value.(yaml.MapSlice)
+		pass := true
+		if !diff(c.Components[0], &exec.FullValue{Elm: elem.Elm}, v[0]) {
+			pass = false
+		}
+		if !diff(c.Components[1], &exec.FullValue{Elm: elem.Elm2}, v[1]) {
+			pass = false
+		}
+		return pass
+
+	case "beam:coder:iterable:v1":
+		pass := true
+		gotrv := reflect.ValueOf(elem.Elm)
+		wantrv := reflect.ValueOf(eg.Value)
+		if gotrv.Len() != wantrv.Len() {
+			log.Printf("Lengths don't match. got %v, want %v;  %v, %v", gotrv.Len(), wantrv.Len(), gotrv, wantrv)
+			return false
+		}
+		for i := 0; i < wantrv.Len(); i++ {
+			if !diff(c.Components[0],
+				&exec.FullValue{Elm: gotrv.Index(i).Interface()},
+				yaml.MapItem{Value: wantrv.Index(i).Interface()}) {
+				pass = false
+			}
+
+		}
+		return pass
+	case "beam:coder:interval_window:v1":
+		var a, b int
+		val := eg.Value
+		if is, ok := eg.Value.([]interface{}); ok {
+			val = is[0]
+		}
+		v := val.(yaml.MapSlice)
+
+		a = v[0].Value.(int)
+		b = v[1].Value.(int)
+		end := mtime.FromMilliseconds(int64(a))
+		start := end - mtime.Time(int64(b))
+		want = window.IntervalWindow{Start: start, End: end}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = elem.Elm
+		} else {
+			got = elem.Windows[0]
+		}
+
+	case "beam:coder:global_window:v1":
+		want = window.GlobalWindow{}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = window.GlobalWindow(elem.Elm.(struct{}))
+		} else {
+			got = elem.Windows[0]
+		}
+	case "beam:coder:windowed_value:v1", "beam:coder:param_windowed_value:v1":
+		// elem contains all the information, but we need to compare the element+timestamp
+		// separately from the windows, to avoid repeated expected value parsing logic.
+		pass := true
+		vs := eg.Value.(yaml.MapSlice)
+		if !diff(c.Components[0], elem, vs[0]) {
+			pass = false
+		}
+		if d := cmp.Diff(
+			mtime.FromMilliseconds(int64(vs[1].Value.(int))),
+			elem.Timestamp, cmpOpts...); d != "" {
+
+			pass = false
+		}
+		if !diff(c.Components[1], elem, vs[3]) {
+			pass = false
+		}
+		// TODO compare pane information.
+		return pass
+	case "beam:coder:row:v1":
+		fs := eg.Value.(yaml.MapSlice)
+		var rfs []reflect.StructField
+		// There are only 2 pointer examples, but they reuse field names,
+		// so we key off the proto hash to know which example we're handling.
+		ptrEg := strings.Contains(c.Payload, "51ace21c7393")
+		for _, rf := range fs {
+			name := rf.Key.(string)
+			t := nameToType[name]
+			if ptrEg {
+				t = reflect.PtrTo(t)
+			}
+			rfs = append(rfs, reflect.StructField{
+				Name: strings.ToUpper(name[:1]) + name[1:],
+				Type: t,
+				Tag:  reflect.StructTag(fmt.Sprintf("beam:\"%v\"", name)),
+			})
+		}
+		rv := reflect.New(reflect.StructOf(rfs)).Elem()
+		for i, rf := range fs {
+			setField(rv, i, rf.Value)
+		}
+
+		got, want = elem.Elm, rv.Interface()
+	default:
+		got, want = elem.Elm, eg.Value
+	}
+	if d := cmp.Diff(want, got, cmpOpts...); d != "" {
+		log.Printf("Decoding error: diff(-want,+got): %v\n", d)
+		return false
+	}
+	return true
+}
+
+// standard_coders.yaml uses the name for type indication, except for nullability.
+var nameToType = map[string]reflect.Type{
+	"str":     reflectx.String,
+	"i32":     reflectx.Int32,
+	"f64":     reflectx.Float64,
+	"arr":     reflect.SliceOf(reflectx.String),
+	"f_bool":  reflectx.Bool,
+	"f_bytes": reflect.PtrTo(reflectx.ByteSlice),
+	"f_map":   reflect.MapOf(reflectx.String, reflect.PtrTo(reflectx.Int64)),
+}
+
+func setField(rv reflect.Value, i int, v interface{}) {
+	if v == nil {
+		return
+	}
+	rf := rv.Field(i)
+	if rf.Kind() == reflect.Ptr {
+		// Ensure it's initialized.
+		rf.Set(reflect.New(rf.Type().Elem()))
+		rf = rf.Elem()
+	}
+	switch rf.Kind() {
+	case reflect.String:
+		rf.SetString(v.(string))
+	case reflect.Int32:
+		rf.SetInt(int64(v.(int)))
+	case reflect.Float64:
+		c, err := strconv.ParseFloat(v.(string), 64)
+		if err != nil {
+			panic(err)
+		}
+		rf.SetFloat(c)
+	case reflect.Slice:
+		if rf.Type() == reflectx.ByteSlice {
+			rf.Set(reflect.ValueOf([]byte(v.(string))))
+			break
+		}
+		// Value is a []interface{} with string values.

Review comment:
       Is this assumption made specifically based on the yaml file this is meant to be used with? Or is this based on something in the schema spec I'm forgetting?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)
+		if c.Custom.Name != "schema" {
+			return dec
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.
+		switch c.T.Type().Kind() {
+		case reflect.Slice:
+			return &lpDecoder{
+				dec: &iterableDecoder{
+					t:   c.Custom.Type,
+					dec: dec,
+				},
+			}
+		case reflect.Array:
+			return &lpDecoder{
+				dec: &arrayDecoder{

Review comment:
       Just to confirm, using an `arrayDecoder` without an `arrayEncoder` specifically works because the format that `iterableEncoder` encodes to is also decodable as an array, right? That might be worth mentioning as a comment, because I was wondering why you can decode arrays with an `arrayDecoder` if they were encoded with an `iterableEncoder`.

##########
File path: sdks/go/pkg/beam/core/runtime/graphx/coder.go
##########
@@ -249,31 +256,35 @@ func (b *CoderUnmarshaller) makeCoder(c *pipepb.Coder) (*coder.Coder, error) {
 			return nil, err
 		}
 
-		// No payload means this coder was length prefixed by the runner
-		// but is likely self describing - AKA a beam coder.
-		if len(sub.GetSpec().GetPayload()) == 0 {
-			return b.makeCoder(sub)
-		}
 		// TODO(lostluck) 2018/10/17: Make this strict again, once dataflow can use
 		// the portable pipeline model directly (BEAM-2885)
-		if sub.GetSpec().GetUrn() != "" && sub.GetSpec().GetUrn() != urnCustomCoder {
-			// TODO(herohde) 11/17/2017: revisit this restriction
-			return nil, errors.Errorf("could not unmarshal length prefix coder from %v, want a custom coder as a sub component but got %v", c, sub)
-		}
-
-		var ref v1pb.CustomCoder
-		if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
-			return nil, err
-		}
-		custom, err := decodeCustomCoder(&ref)
-		if err != nil {
-			return nil, err
+		switch u := sub.GetSpec().GetUrn(); u {
+		case "", urnCustomCoder:
+			var ref v1pb.CustomCoder
+			if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
+				return nil, err
+			}
+			custom, err := decodeCustomCoder(&ref)
+			if err != nil {
+				return nil, err
+			}
+			custom.ID = components[0]
+			t := typex.New(custom.Type)
+			cc := &coder.Coder{Kind: coder.Custom, T: t, Custom: custom}
+			fmt.Println("decoded customcoder", cc)

Review comment:
       Looks like a debug Println left in.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)

Review comment:
       Is it necessary to cast n to an int here? Isn't it already an int from line 638?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {
+		err := c.dec.DecodeTo(r, &e)
+		if err != nil {
+			return reflect.Value{}, err
+		}
+		if e.Elm != nil {
+			rv.Index(i).Set(reflect.ValueOf(e.Elm))
+		} else {
+			rv.Index(i).Set(reflect.ValueOf(e.Windows[0]))

Review comment:
       I don't get why it's setting the value of the element to `e.Windows[0]`. Is it a way to pass in nil?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {

Review comment:
       Likewise, is it necessary to cast n here too?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -81,24 +82,82 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder {
 		return &stringEncoder{}
 
 	case coder.Custom:
-		return &customEncoder{
+		enc := &customEncoder{
 			t:   c.Custom.Type,
 			enc: makeEncoder(c.Custom.Enc.Fn),
 		}
+		if c.Custom.Name != "schema" {
+			return enc
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.

Review comment:
       I don't really understand this comment, or how it relates to the check below.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)

Review comment:
       Looks like another debug println.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -534,26 +1038,34 @@ func (*intervalWindowEncoder) EncodeSingle(elm typex.Window, w io.Writer) error
 
 type intervalWindowDecoder struct{}
 
-func (*intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
+func (d *intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
 	// Encoding: upper bound and duration
 
 	n, err := coder.DecodeInt32(r) // #windows
 
 	ret := make([]typex.Window, n, n)
 	for i := int32(0); i < n; i++ {
-		end, err := coder.DecodeEventTime(r)
-		if err != nil {
-			return nil, err
-		}
-		duration, err := coder.DecodeVarUint64(r)
+		w, err := d.DecodeSingle(r)
 		if err != nil {
 			return nil, err
 		}
-		ret[i] = window.IntervalWindow{Start: mtime.FromMilliseconds(end.Milliseconds() - int64(duration)), End: end}
+		ret[i] = w
 	}
 	return ret, err
 }
 
+func (*intervalWindowDecoder) DecodeSingle(r io.Reader) (typex.Window, error) {
+	end, err := coder.DecodeEventTime(r)
+	if err != nil {
+		return nil, err
+	}
+	duration, err := coder.DecodeVarInt(r)

Review comment:
       What's the reason to switch from `DecodeVarUint64` to `DecodeVarInt` here?

##########
File path: sdks/go/test/regression/coders/fromyaml/fromyaml.go
##########
@@ -0,0 +1,415 @@
+// 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.
+
+// fromyaml generates a resource file from the standard_coders.yaml
+// file for use in these coder regression tests.
+//
+// It expects to be run in it's test directory, or via it's go test.
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"math"
+	"reflect"
+	"runtime/debug"
+	"strconv"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/google/go-cmp/cmp"
+	"golang.org/x/text/encoding/charmap"
+	yaml "gopkg.in/yaml.v2"
+)
+
+var unimplementedCoders = map[string]bool{
+	"beam:coder:param_windowed_value:v1": true,
+	"beam:coder:timer:v1":                true,
+}
+
+// Coder is a representation a serialized beam coder.
+type Coder struct {
+	Urn              string  `yaml:"urn,omitempty"`
+	Payload          string  `yaml:"payload,omitempty"`
+	Components       []Coder `yaml:"components,omitempty"`
+	NonDeterministic bool    `yaml:"non_deterministic,omitempty"`
+}
+
+type logger interface {
+	Errorf(string, ...interface{})
+	Logf(string, ...interface{})
+}
+
+// Spec is a set of conditions that a coder must pass.
+type Spec struct {
+	Coder    Coder         `yaml:"coder,omitempty"`
+	Nested   *bool         `yaml:"nested,omitempty"`
+	Examples yaml.MapSlice `yaml:"examples,omitempty"`
+	Log      logger
+
+	id       int // for generating coder ids.
+	coderPBs map[string]*pipepb.Coder
+}
+
+func (s *Spec) nextID() string {
+	ret := fmt.Sprintf("%d", s.id)
+	s.id++
+	return ret
+}
+
+func (s *Spec) testStandardCoder() (err error) {
+	if unimplementedCoders[s.Coder.Urn] {
+		log.Printf("skipping unimplemented coder urn: %v", s.Coder.Urn)
+		return nil
+	}
+	// Construct the coder proto equivalents.
+
+	// Only nested tests need to be run, since nestedness is a pre-portability
+	// concept.
+	// For legacy Java reasons, the row coder examples are all marked nested: false
+	// so we need to check that before skipping unnested tests.
+	if s.Coder.Urn != "beam:coder:row:v1" && s.Nested != nil && !*s.Nested {
+		log.Printf("skipping unnested coder spec: %v\n", s.Coder)
+		return nil
+	}
+
+	s.coderPBs = make(map[string]*pipepb.Coder)
+	id := s.parseCoder(s.Coder)
+	b := graphx.NewCoderUnmarshaller(s.coderPBs)
+	underTest, err := b.Coder(id)
+	if err != nil {
+		return fmt.Errorf("unable to create coder: %v", err)
+	}
+
+	defer func() {
+		if e := recover(); e != nil {
+			err = fmt.Errorf("panicked on coder %v || %v:\n\t%v :\n%s", underTest, s.Coder, e, debug.Stack())
+		}
+	}()
+
+	var decFails, encFails int
+	for _, eg := range s.Examples {
+
+		// Test Decoding
+		// Ideally we'd use the beam package coders, but KVs make that complicated.
+		// This can be cleaned up once a type parametered beam.KV type exists.
+		dec := exec.MakeElementDecoder(underTest)
+		encoded := eg.Key.(string)
+		var elem exec.FullValue
+
+		// What I would have expected.
+		//		r := charmap.ISO8859_1.NewDecoder().Reader(strings.NewReader(encoded))
+		recoded, err := charmap.ISO8859_1.NewEncoder().String(encoded)
+		if err != nil {
+			return err
+		}
+		r := strings.NewReader(recoded)
+		if err := dec.DecodeTo(r, &elem); err != nil {
+			return fmt.Errorf("err decoding %q: %v", encoded, err)
+		}
+		if !diff(s.Coder, &elem, eg) {
+			decFails++
+			continue
+		}
+
+		// Test Encoding
+		if s.Coder.NonDeterministic {
+			// Skip verifying nondeterministic encodings.
+			continue
+		}
+		enc := exec.MakeElementEncoder(underTest)
+		var out bytes.Buffer
+		if err := enc.Encode(&elem, &out); err != nil {
+			return err
+		}
+		if d := cmp.Diff(recoded, string(out.Bytes())); d != "" {
+			log.Printf("Encoding error: diff(-want,+got): %v\n", d)
+		}
+	}
+	if decFails+encFails > 0 {
+		return fmt.Errorf("failed to decode %v times, and encode %v times", decFails, encFails)
+	}
+
+	return nil
+}
+
+var cmpOpts = []cmp.Option{
+	cmp.Transformer("bytes2string", func(in []byte) (out string) {
+		return string(in)
+	}),
+}
+
+func diff(c Coder, elem *exec.FullValue, eg yaml.MapItem) bool {
+	var got, want interface{}
+	switch c.Urn {
+	case "beam:coder:bytes:v1":
+		got = string(elem.Elm.([]byte))
+		switch egv := eg.Value.(type) {
+		case string:
+			want = egv
+		case []byte:
+			want = string(egv)
+		}
+	case "beam:coder:varint:v1":
+		got, want = elem.Elm.(int64), int64(eg.Value.(int))
+	case "beam:coder:double:v1":
+		got = elem.Elm.(float64)
+		switch v := eg.Value.(string); v {
+		case "NaN":
+			// Do the NaN comparison here since NaN by definition != NaN.
+			if math.IsNaN(got.(float64)) {
+				want, got = 1, 1
+			} else {
+				want = math.NaN()
+			}
+		case "-Infinity":
+			want = math.Inf(-1)
+		case "Infinity":
+			want = math.Inf(1)
+		default:
+			want, _ = strconv.ParseFloat(v, 64)
+		}
+
+	case "beam:coder:kv:v1":
+		v := eg.Value.(yaml.MapSlice)
+		pass := true
+		if !diff(c.Components[0], &exec.FullValue{Elm: elem.Elm}, v[0]) {
+			pass = false
+		}
+		if !diff(c.Components[1], &exec.FullValue{Elm: elem.Elm2}, v[1]) {
+			pass = false
+		}
+		return pass
+
+	case "beam:coder:iterable:v1":
+		pass := true
+		gotrv := reflect.ValueOf(elem.Elm)
+		wantrv := reflect.ValueOf(eg.Value)
+		if gotrv.Len() != wantrv.Len() {
+			log.Printf("Lengths don't match. got %v, want %v;  %v, %v", gotrv.Len(), wantrv.Len(), gotrv, wantrv)
+			return false
+		}
+		for i := 0; i < wantrv.Len(); i++ {
+			if !diff(c.Components[0],
+				&exec.FullValue{Elm: gotrv.Index(i).Interface()},
+				yaml.MapItem{Value: wantrv.Index(i).Interface()}) {
+				pass = false
+			}
+
+		}
+		return pass
+	case "beam:coder:interval_window:v1":
+		var a, b int
+		val := eg.Value
+		if is, ok := eg.Value.([]interface{}); ok {
+			val = is[0]
+		}
+		v := val.(yaml.MapSlice)
+
+		a = v[0].Value.(int)
+		b = v[1].Value.(int)
+		end := mtime.FromMilliseconds(int64(a))
+		start := end - mtime.Time(int64(b))
+		want = window.IntervalWindow{Start: start, End: end}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = elem.Elm
+		} else {
+			got = elem.Windows[0]
+		}
+
+	case "beam:coder:global_window:v1":
+		want = window.GlobalWindow{}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = window.GlobalWindow(elem.Elm.(struct{}))
+		} else {
+			got = elem.Windows[0]
+		}
+	case "beam:coder:windowed_value:v1", "beam:coder:param_windowed_value:v1":
+		// elem contains all the information, but we need to compare the element+timestamp
+		// separately from the windows, to avoid repeated expected value parsing logic.
+		pass := true
+		vs := eg.Value.(yaml.MapSlice)
+		if !diff(c.Components[0], elem, vs[0]) {
+			pass = false
+		}
+		if d := cmp.Diff(
+			mtime.FromMilliseconds(int64(vs[1].Value.(int))),
+			elem.Timestamp, cmpOpts...); d != "" {
+
+			pass = false
+		}
+		if !diff(c.Components[1], elem, vs[3]) {
+			pass = false
+		}
+		// TODO compare pane information.
+		return pass
+	case "beam:coder:row:v1":
+		fs := eg.Value.(yaml.MapSlice)
+		var rfs []reflect.StructField
+		// There are only 2 pointer examples, but they reuse field names,
+		// so we key off the proto hash to know which example we're handling.
+		ptrEg := strings.Contains(c.Payload, "51ace21c7393")
+		for _, rf := range fs {
+			name := rf.Key.(string)
+			t := nameToType[name]
+			if ptrEg {
+				t = reflect.PtrTo(t)
+			}
+			rfs = append(rfs, reflect.StructField{
+				Name: strings.ToUpper(name[:1]) + name[1:],
+				Type: t,
+				Tag:  reflect.StructTag(fmt.Sprintf("beam:\"%v\"", name)),
+			})
+		}
+		rv := reflect.New(reflect.StructOf(rfs)).Elem()
+		for i, rf := range fs {
+			setField(rv, i, rf.Value)
+		}
+
+		got, want = elem.Elm, rv.Interface()
+	default:
+		got, want = elem.Elm, eg.Value
+	}
+	if d := cmp.Diff(want, got, cmpOpts...); d != "" {
+		log.Printf("Decoding error: diff(-want,+got): %v\n", d)
+		return false
+	}
+	return true
+}
+
+// standard_coders.yaml uses the name for type indication, except for nullability.
+var nameToType = map[string]reflect.Type{
+	"str":     reflectx.String,
+	"i32":     reflectx.Int32,
+	"f64":     reflectx.Float64,
+	"arr":     reflect.SliceOf(reflectx.String),
+	"f_bool":  reflectx.Bool,
+	"f_bytes": reflect.PtrTo(reflectx.ByteSlice),
+	"f_map":   reflect.MapOf(reflectx.String, reflect.PtrTo(reflectx.Int64)),
+}
+
+func setField(rv reflect.Value, i int, v interface{}) {
+	if v == nil {
+		return
+	}
+	rf := rv.Field(i)
+	if rf.Kind() == reflect.Ptr {
+		// Ensure it's initialized.
+		rf.Set(reflect.New(rf.Type().Elem()))
+		rf = rf.Elem()
+	}
+	switch rf.Kind() {
+	case reflect.String:
+		rf.SetString(v.(string))
+	case reflect.Int32:
+		rf.SetInt(int64(v.(int)))
+	case reflect.Float64:
+		c, err := strconv.ParseFloat(v.(string), 64)
+		if err != nil {
+			panic(err)
+		}
+		rf.SetFloat(c)
+	case reflect.Slice:
+		if rf.Type() == reflectx.ByteSlice {
+			rf.Set(reflect.ValueOf([]byte(v.(string))))
+			break
+		}
+		// Value is a []interface{} with string values.

Review comment:
       Is this assumption made specifically based on the yaml file this is meant to be used with? Or is this based on something in the schema spec I'm forgetting?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)
+		if c.Custom.Name != "schema" {
+			return dec
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.
+		switch c.T.Type().Kind() {
+		case reflect.Slice:
+			return &lpDecoder{
+				dec: &iterableDecoder{
+					t:   c.Custom.Type,
+					dec: dec,
+				},
+			}
+		case reflect.Array:
+			return &lpDecoder{
+				dec: &arrayDecoder{

Review comment:
       Just to confirm, using an `arrayDecoder` without an `arrayEncoder` specifically works because the format that `iterableEncoder` encodes to is also decodable as an array, right? That might be worth mentioning as a comment, because I was wondering why you can decode arrays with an `arrayDecoder` if they were encoded with an `iterableEncoder`.

##########
File path: sdks/go/pkg/beam/core/runtime/graphx/coder.go
##########
@@ -249,31 +256,35 @@ func (b *CoderUnmarshaller) makeCoder(c *pipepb.Coder) (*coder.Coder, error) {
 			return nil, err
 		}
 
-		// No payload means this coder was length prefixed by the runner
-		// but is likely self describing - AKA a beam coder.
-		if len(sub.GetSpec().GetPayload()) == 0 {
-			return b.makeCoder(sub)
-		}
 		// TODO(lostluck) 2018/10/17: Make this strict again, once dataflow can use
 		// the portable pipeline model directly (BEAM-2885)
-		if sub.GetSpec().GetUrn() != "" && sub.GetSpec().GetUrn() != urnCustomCoder {
-			// TODO(herohde) 11/17/2017: revisit this restriction
-			return nil, errors.Errorf("could not unmarshal length prefix coder from %v, want a custom coder as a sub component but got %v", c, sub)
-		}
-
-		var ref v1pb.CustomCoder
-		if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
-			return nil, err
-		}
-		custom, err := decodeCustomCoder(&ref)
-		if err != nil {
-			return nil, err
+		switch u := sub.GetSpec().GetUrn(); u {
+		case "", urnCustomCoder:
+			var ref v1pb.CustomCoder
+			if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
+				return nil, err
+			}
+			custom, err := decodeCustomCoder(&ref)
+			if err != nil {
+				return nil, err
+			}
+			custom.ID = components[0]
+			t := typex.New(custom.Type)
+			cc := &coder.Coder{Kind: coder.Custom, T: t, Custom: custom}
+			fmt.Println("decoded customcoder", cc)

Review comment:
       Looks like a debug Println left in.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)

Review comment:
       Is it necessary to cast n to an int here? Isn't it already an int from line 638?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {
+		err := c.dec.DecodeTo(r, &e)
+		if err != nil {
+			return reflect.Value{}, err
+		}
+		if e.Elm != nil {
+			rv.Index(i).Set(reflect.ValueOf(e.Elm))
+		} else {
+			rv.Index(i).Set(reflect.ValueOf(e.Windows[0]))

Review comment:
       I don't get why it's setting the value of the element to `e.Windows[0]`. Is it a way to pass in nil?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {

Review comment:
       Likewise, is it necessary to cast n here too?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -81,24 +82,82 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder {
 		return &stringEncoder{}
 
 	case coder.Custom:
-		return &customEncoder{
+		enc := &customEncoder{
 			t:   c.Custom.Type,
 			enc: makeEncoder(c.Custom.Enc.Fn),
 		}
+		if c.Custom.Name != "schema" {
+			return enc
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.

Review comment:
       I don't really understand this comment, or how it relates to the check below.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)

Review comment:
       Looks like another debug println.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -534,26 +1038,34 @@ func (*intervalWindowEncoder) EncodeSingle(elm typex.Window, w io.Writer) error
 
 type intervalWindowDecoder struct{}
 
-func (*intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
+func (d *intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
 	// Encoding: upper bound and duration
 
 	n, err := coder.DecodeInt32(r) // #windows
 
 	ret := make([]typex.Window, n, n)
 	for i := int32(0); i < n; i++ {
-		end, err := coder.DecodeEventTime(r)
-		if err != nil {
-			return nil, err
-		}
-		duration, err := coder.DecodeVarUint64(r)
+		w, err := d.DecodeSingle(r)
 		if err != nil {
 			return nil, err
 		}
-		ret[i] = window.IntervalWindow{Start: mtime.FromMilliseconds(end.Milliseconds() - int64(duration)), End: end}
+		ret[i] = w
 	}
 	return ret, err
 }
 
+func (*intervalWindowDecoder) DecodeSingle(r io.Reader) (typex.Window, error) {
+	end, err := coder.DecodeEventTime(r)
+	if err != nil {
+		return nil, err
+	}
+	duration, err := coder.DecodeVarInt(r)

Review comment:
       What's the reason to switch from `DecodeVarUint64` to `DecodeVarInt` here?

##########
File path: sdks/go/test/regression/coders/fromyaml/fromyaml.go
##########
@@ -0,0 +1,415 @@
+// 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.
+
+// fromyaml generates a resource file from the standard_coders.yaml
+// file for use in these coder regression tests.
+//
+// It expects to be run in it's test directory, or via it's go test.
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"math"
+	"reflect"
+	"runtime/debug"
+	"strconv"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/google/go-cmp/cmp"
+	"golang.org/x/text/encoding/charmap"
+	yaml "gopkg.in/yaml.v2"
+)
+
+var unimplementedCoders = map[string]bool{
+	"beam:coder:param_windowed_value:v1": true,
+	"beam:coder:timer:v1":                true,
+}
+
+// Coder is a representation a serialized beam coder.
+type Coder struct {
+	Urn              string  `yaml:"urn,omitempty"`
+	Payload          string  `yaml:"payload,omitempty"`
+	Components       []Coder `yaml:"components,omitempty"`
+	NonDeterministic bool    `yaml:"non_deterministic,omitempty"`
+}
+
+type logger interface {
+	Errorf(string, ...interface{})
+	Logf(string, ...interface{})
+}
+
+// Spec is a set of conditions that a coder must pass.
+type Spec struct {
+	Coder    Coder         `yaml:"coder,omitempty"`
+	Nested   *bool         `yaml:"nested,omitempty"`
+	Examples yaml.MapSlice `yaml:"examples,omitempty"`
+	Log      logger
+
+	id       int // for generating coder ids.
+	coderPBs map[string]*pipepb.Coder
+}
+
+func (s *Spec) nextID() string {
+	ret := fmt.Sprintf("%d", s.id)
+	s.id++
+	return ret
+}
+
+func (s *Spec) testStandardCoder() (err error) {
+	if unimplementedCoders[s.Coder.Urn] {
+		log.Printf("skipping unimplemented coder urn: %v", s.Coder.Urn)
+		return nil
+	}
+	// Construct the coder proto equivalents.
+
+	// Only nested tests need to be run, since nestedness is a pre-portability
+	// concept.
+	// For legacy Java reasons, the row coder examples are all marked nested: false
+	// so we need to check that before skipping unnested tests.
+	if s.Coder.Urn != "beam:coder:row:v1" && s.Nested != nil && !*s.Nested {
+		log.Printf("skipping unnested coder spec: %v\n", s.Coder)
+		return nil
+	}
+
+	s.coderPBs = make(map[string]*pipepb.Coder)
+	id := s.parseCoder(s.Coder)
+	b := graphx.NewCoderUnmarshaller(s.coderPBs)
+	underTest, err := b.Coder(id)
+	if err != nil {
+		return fmt.Errorf("unable to create coder: %v", err)
+	}
+
+	defer func() {
+		if e := recover(); e != nil {
+			err = fmt.Errorf("panicked on coder %v || %v:\n\t%v :\n%s", underTest, s.Coder, e, debug.Stack())
+		}
+	}()
+
+	var decFails, encFails int
+	for _, eg := range s.Examples {
+
+		// Test Decoding
+		// Ideally we'd use the beam package coders, but KVs make that complicated.
+		// This can be cleaned up once a type parametered beam.KV type exists.
+		dec := exec.MakeElementDecoder(underTest)
+		encoded := eg.Key.(string)
+		var elem exec.FullValue
+
+		// What I would have expected.
+		//		r := charmap.ISO8859_1.NewDecoder().Reader(strings.NewReader(encoded))
+		recoded, err := charmap.ISO8859_1.NewEncoder().String(encoded)
+		if err != nil {
+			return err
+		}
+		r := strings.NewReader(recoded)
+		if err := dec.DecodeTo(r, &elem); err != nil {
+			return fmt.Errorf("err decoding %q: %v", encoded, err)
+		}
+		if !diff(s.Coder, &elem, eg) {
+			decFails++
+			continue
+		}
+
+		// Test Encoding
+		if s.Coder.NonDeterministic {
+			// Skip verifying nondeterministic encodings.
+			continue
+		}
+		enc := exec.MakeElementEncoder(underTest)
+		var out bytes.Buffer
+		if err := enc.Encode(&elem, &out); err != nil {
+			return err
+		}
+		if d := cmp.Diff(recoded, string(out.Bytes())); d != "" {
+			log.Printf("Encoding error: diff(-want,+got): %v\n", d)
+		}
+	}
+	if decFails+encFails > 0 {
+		return fmt.Errorf("failed to decode %v times, and encode %v times", decFails, encFails)
+	}
+
+	return nil
+}
+
+var cmpOpts = []cmp.Option{
+	cmp.Transformer("bytes2string", func(in []byte) (out string) {
+		return string(in)
+	}),
+}
+
+func diff(c Coder, elem *exec.FullValue, eg yaml.MapItem) bool {
+	var got, want interface{}
+	switch c.Urn {
+	case "beam:coder:bytes:v1":
+		got = string(elem.Elm.([]byte))
+		switch egv := eg.Value.(type) {
+		case string:
+			want = egv
+		case []byte:
+			want = string(egv)
+		}
+	case "beam:coder:varint:v1":
+		got, want = elem.Elm.(int64), int64(eg.Value.(int))
+	case "beam:coder:double:v1":
+		got = elem.Elm.(float64)
+		switch v := eg.Value.(string); v {
+		case "NaN":
+			// Do the NaN comparison here since NaN by definition != NaN.
+			if math.IsNaN(got.(float64)) {
+				want, got = 1, 1
+			} else {
+				want = math.NaN()
+			}
+		case "-Infinity":
+			want = math.Inf(-1)
+		case "Infinity":
+			want = math.Inf(1)
+		default:
+			want, _ = strconv.ParseFloat(v, 64)
+		}
+
+	case "beam:coder:kv:v1":
+		v := eg.Value.(yaml.MapSlice)
+		pass := true
+		if !diff(c.Components[0], &exec.FullValue{Elm: elem.Elm}, v[0]) {
+			pass = false
+		}
+		if !diff(c.Components[1], &exec.FullValue{Elm: elem.Elm2}, v[1]) {
+			pass = false
+		}
+		return pass
+
+	case "beam:coder:iterable:v1":
+		pass := true
+		gotrv := reflect.ValueOf(elem.Elm)
+		wantrv := reflect.ValueOf(eg.Value)
+		if gotrv.Len() != wantrv.Len() {
+			log.Printf("Lengths don't match. got %v, want %v;  %v, %v", gotrv.Len(), wantrv.Len(), gotrv, wantrv)
+			return false
+		}
+		for i := 0; i < wantrv.Len(); i++ {
+			if !diff(c.Components[0],
+				&exec.FullValue{Elm: gotrv.Index(i).Interface()},
+				yaml.MapItem{Value: wantrv.Index(i).Interface()}) {
+				pass = false
+			}
+
+		}
+		return pass
+	case "beam:coder:interval_window:v1":
+		var a, b int
+		val := eg.Value
+		if is, ok := eg.Value.([]interface{}); ok {
+			val = is[0]
+		}
+		v := val.(yaml.MapSlice)
+
+		a = v[0].Value.(int)
+		b = v[1].Value.(int)
+		end := mtime.FromMilliseconds(int64(a))
+		start := end - mtime.Time(int64(b))
+		want = window.IntervalWindow{Start: start, End: end}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = elem.Elm
+		} else {
+			got = elem.Windows[0]
+		}
+
+	case "beam:coder:global_window:v1":
+		want = window.GlobalWindow{}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = window.GlobalWindow(elem.Elm.(struct{}))
+		} else {
+			got = elem.Windows[0]
+		}
+	case "beam:coder:windowed_value:v1", "beam:coder:param_windowed_value:v1":
+		// elem contains all the information, but we need to compare the element+timestamp
+		// separately from the windows, to avoid repeated expected value parsing logic.
+		pass := true
+		vs := eg.Value.(yaml.MapSlice)
+		if !diff(c.Components[0], elem, vs[0]) {
+			pass = false
+		}
+		if d := cmp.Diff(
+			mtime.FromMilliseconds(int64(vs[1].Value.(int))),
+			elem.Timestamp, cmpOpts...); d != "" {
+
+			pass = false
+		}
+		if !diff(c.Components[1], elem, vs[3]) {
+			pass = false
+		}
+		// TODO compare pane information.
+		return pass
+	case "beam:coder:row:v1":
+		fs := eg.Value.(yaml.MapSlice)
+		var rfs []reflect.StructField
+		// There are only 2 pointer examples, but they reuse field names,
+		// so we key off the proto hash to know which example we're handling.
+		ptrEg := strings.Contains(c.Payload, "51ace21c7393")
+		for _, rf := range fs {
+			name := rf.Key.(string)
+			t := nameToType[name]
+			if ptrEg {
+				t = reflect.PtrTo(t)
+			}
+			rfs = append(rfs, reflect.StructField{
+				Name: strings.ToUpper(name[:1]) + name[1:],
+				Type: t,
+				Tag:  reflect.StructTag(fmt.Sprintf("beam:\"%v\"", name)),
+			})
+		}
+		rv := reflect.New(reflect.StructOf(rfs)).Elem()
+		for i, rf := range fs {
+			setField(rv, i, rf.Value)
+		}
+
+		got, want = elem.Elm, rv.Interface()
+	default:
+		got, want = elem.Elm, eg.Value
+	}
+	if d := cmp.Diff(want, got, cmpOpts...); d != "" {
+		log.Printf("Decoding error: diff(-want,+got): %v\n", d)
+		return false
+	}
+	return true
+}
+
+// standard_coders.yaml uses the name for type indication, except for nullability.
+var nameToType = map[string]reflect.Type{
+	"str":     reflectx.String,
+	"i32":     reflectx.Int32,
+	"f64":     reflectx.Float64,
+	"arr":     reflect.SliceOf(reflectx.String),
+	"f_bool":  reflectx.Bool,
+	"f_bytes": reflect.PtrTo(reflectx.ByteSlice),
+	"f_map":   reflect.MapOf(reflectx.String, reflect.PtrTo(reflectx.Int64)),
+}
+
+func setField(rv reflect.Value, i int, v interface{}) {
+	if v == nil {
+		return
+	}
+	rf := rv.Field(i)
+	if rf.Kind() == reflect.Ptr {
+		// Ensure it's initialized.
+		rf.Set(reflect.New(rf.Type().Elem()))
+		rf = rf.Elem()
+	}
+	switch rf.Kind() {
+	case reflect.String:
+		rf.SetString(v.(string))
+	case reflect.Int32:
+		rf.SetInt(int64(v.(int)))
+	case reflect.Float64:
+		c, err := strconv.ParseFloat(v.(string), 64)
+		if err != nil {
+			panic(err)
+		}
+		rf.SetFloat(c)
+	case reflect.Slice:
+		if rf.Type() == reflectx.ByteSlice {
+			rf.Set(reflect.ValueOf([]byte(v.(string))))
+			break
+		}
+		// Value is a []interface{} with string values.

Review comment:
       Is this assumption made specifically based on the yaml file this is meant to be used with? Or is this based on something in the schema spec I'm forgetting?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)
+		if c.Custom.Name != "schema" {
+			return dec
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.
+		switch c.T.Type().Kind() {
+		case reflect.Slice:
+			return &lpDecoder{
+				dec: &iterableDecoder{
+					t:   c.Custom.Type,
+					dec: dec,
+				},
+			}
+		case reflect.Array:
+			return &lpDecoder{
+				dec: &arrayDecoder{

Review comment:
       Just to confirm, using an `arrayDecoder` without an `arrayEncoder` specifically works because the format that `iterableEncoder` encodes to is also decodable as an array, right? That might be worth mentioning as a comment, because I was wondering why you can decode arrays with an `arrayDecoder` if they were encoded with an `iterableEncoder`.

##########
File path: sdks/go/pkg/beam/core/runtime/graphx/coder.go
##########
@@ -249,31 +256,35 @@ func (b *CoderUnmarshaller) makeCoder(c *pipepb.Coder) (*coder.Coder, error) {
 			return nil, err
 		}
 
-		// No payload means this coder was length prefixed by the runner
-		// but is likely self describing - AKA a beam coder.
-		if len(sub.GetSpec().GetPayload()) == 0 {
-			return b.makeCoder(sub)
-		}
 		// TODO(lostluck) 2018/10/17: Make this strict again, once dataflow can use
 		// the portable pipeline model directly (BEAM-2885)
-		if sub.GetSpec().GetUrn() != "" && sub.GetSpec().GetUrn() != urnCustomCoder {
-			// TODO(herohde) 11/17/2017: revisit this restriction
-			return nil, errors.Errorf("could not unmarshal length prefix coder from %v, want a custom coder as a sub component but got %v", c, sub)
-		}
-
-		var ref v1pb.CustomCoder
-		if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
-			return nil, err
-		}
-		custom, err := decodeCustomCoder(&ref)
-		if err != nil {
-			return nil, err
+		switch u := sub.GetSpec().GetUrn(); u {
+		case "", urnCustomCoder:
+			var ref v1pb.CustomCoder
+			if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
+				return nil, err
+			}
+			custom, err := decodeCustomCoder(&ref)
+			if err != nil {
+				return nil, err
+			}
+			custom.ID = components[0]
+			t := typex.New(custom.Type)
+			cc := &coder.Coder{Kind: coder.Custom, T: t, Custom: custom}
+			fmt.Println("decoded customcoder", cc)

Review comment:
       Looks like a debug Println left in.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)

Review comment:
       Is it necessary to cast n to an int here? Isn't it already an int from line 638?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {
+		err := c.dec.DecodeTo(r, &e)
+		if err != nil {
+			return reflect.Value{}, err
+		}
+		if e.Elm != nil {
+			rv.Index(i).Set(reflect.ValueOf(e.Elm))
+		} else {
+			rv.Index(i).Set(reflect.ValueOf(e.Windows[0]))

Review comment:
       I don't get why it's setting the value of the element to `e.Windows[0]`. Is it a way to pass in nil?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {

Review comment:
       Likewise, is it necessary to cast n here too?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -81,24 +82,82 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder {
 		return &stringEncoder{}
 
 	case coder.Custom:
-		return &customEncoder{
+		enc := &customEncoder{
 			t:   c.Custom.Type,
 			enc: makeEncoder(c.Custom.Enc.Fn),
 		}
+		if c.Custom.Name != "schema" {
+			return enc
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.

Review comment:
       I don't really understand this comment, or how it relates to the check below.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)

Review comment:
       Looks like another debug println.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -534,26 +1038,34 @@ func (*intervalWindowEncoder) EncodeSingle(elm typex.Window, w io.Writer) error
 
 type intervalWindowDecoder struct{}
 
-func (*intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
+func (d *intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
 	// Encoding: upper bound and duration
 
 	n, err := coder.DecodeInt32(r) // #windows
 
 	ret := make([]typex.Window, n, n)
 	for i := int32(0); i < n; i++ {
-		end, err := coder.DecodeEventTime(r)
-		if err != nil {
-			return nil, err
-		}
-		duration, err := coder.DecodeVarUint64(r)
+		w, err := d.DecodeSingle(r)
 		if err != nil {
 			return nil, err
 		}
-		ret[i] = window.IntervalWindow{Start: mtime.FromMilliseconds(end.Milliseconds() - int64(duration)), End: end}
+		ret[i] = w
 	}
 	return ret, err
 }
 
+func (*intervalWindowDecoder) DecodeSingle(r io.Reader) (typex.Window, error) {
+	end, err := coder.DecodeEventTime(r)
+	if err != nil {
+		return nil, err
+	}
+	duration, err := coder.DecodeVarInt(r)

Review comment:
       What's the reason to switch from `DecodeVarUint64` to `DecodeVarInt` here?

##########
File path: sdks/go/test/regression/coders/fromyaml/fromyaml.go
##########
@@ -0,0 +1,415 @@
+// 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.
+
+// fromyaml generates a resource file from the standard_coders.yaml
+// file for use in these coder regression tests.
+//
+// It expects to be run in it's test directory, or via it's go test.
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"math"
+	"reflect"
+	"runtime/debug"
+	"strconv"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/google/go-cmp/cmp"
+	"golang.org/x/text/encoding/charmap"
+	yaml "gopkg.in/yaml.v2"
+)
+
+var unimplementedCoders = map[string]bool{
+	"beam:coder:param_windowed_value:v1": true,
+	"beam:coder:timer:v1":                true,
+}
+
+// Coder is a representation a serialized beam coder.
+type Coder struct {
+	Urn              string  `yaml:"urn,omitempty"`
+	Payload          string  `yaml:"payload,omitempty"`
+	Components       []Coder `yaml:"components,omitempty"`
+	NonDeterministic bool    `yaml:"non_deterministic,omitempty"`
+}
+
+type logger interface {
+	Errorf(string, ...interface{})
+	Logf(string, ...interface{})
+}
+
+// Spec is a set of conditions that a coder must pass.
+type Spec struct {
+	Coder    Coder         `yaml:"coder,omitempty"`
+	Nested   *bool         `yaml:"nested,omitempty"`
+	Examples yaml.MapSlice `yaml:"examples,omitempty"`
+	Log      logger
+
+	id       int // for generating coder ids.
+	coderPBs map[string]*pipepb.Coder
+}
+
+func (s *Spec) nextID() string {
+	ret := fmt.Sprintf("%d", s.id)
+	s.id++
+	return ret
+}
+
+func (s *Spec) testStandardCoder() (err error) {
+	if unimplementedCoders[s.Coder.Urn] {
+		log.Printf("skipping unimplemented coder urn: %v", s.Coder.Urn)
+		return nil
+	}
+	// Construct the coder proto equivalents.
+
+	// Only nested tests need to be run, since nestedness is a pre-portability
+	// concept.
+	// For legacy Java reasons, the row coder examples are all marked nested: false
+	// so we need to check that before skipping unnested tests.
+	if s.Coder.Urn != "beam:coder:row:v1" && s.Nested != nil && !*s.Nested {
+		log.Printf("skipping unnested coder spec: %v\n", s.Coder)
+		return nil
+	}
+
+	s.coderPBs = make(map[string]*pipepb.Coder)
+	id := s.parseCoder(s.Coder)
+	b := graphx.NewCoderUnmarshaller(s.coderPBs)
+	underTest, err := b.Coder(id)
+	if err != nil {
+		return fmt.Errorf("unable to create coder: %v", err)
+	}
+
+	defer func() {
+		if e := recover(); e != nil {
+			err = fmt.Errorf("panicked on coder %v || %v:\n\t%v :\n%s", underTest, s.Coder, e, debug.Stack())
+		}
+	}()
+
+	var decFails, encFails int
+	for _, eg := range s.Examples {
+
+		// Test Decoding
+		// Ideally we'd use the beam package coders, but KVs make that complicated.
+		// This can be cleaned up once a type parametered beam.KV type exists.
+		dec := exec.MakeElementDecoder(underTest)
+		encoded := eg.Key.(string)
+		var elem exec.FullValue
+
+		// What I would have expected.
+		//		r := charmap.ISO8859_1.NewDecoder().Reader(strings.NewReader(encoded))
+		recoded, err := charmap.ISO8859_1.NewEncoder().String(encoded)
+		if err != nil {
+			return err
+		}
+		r := strings.NewReader(recoded)
+		if err := dec.DecodeTo(r, &elem); err != nil {
+			return fmt.Errorf("err decoding %q: %v", encoded, err)
+		}
+		if !diff(s.Coder, &elem, eg) {
+			decFails++
+			continue
+		}
+
+		// Test Encoding
+		if s.Coder.NonDeterministic {
+			// Skip verifying nondeterministic encodings.
+			continue
+		}
+		enc := exec.MakeElementEncoder(underTest)
+		var out bytes.Buffer
+		if err := enc.Encode(&elem, &out); err != nil {
+			return err
+		}
+		if d := cmp.Diff(recoded, string(out.Bytes())); d != "" {
+			log.Printf("Encoding error: diff(-want,+got): %v\n", d)
+		}
+	}
+	if decFails+encFails > 0 {
+		return fmt.Errorf("failed to decode %v times, and encode %v times", decFails, encFails)
+	}
+
+	return nil
+}
+
+var cmpOpts = []cmp.Option{
+	cmp.Transformer("bytes2string", func(in []byte) (out string) {
+		return string(in)
+	}),
+}
+
+func diff(c Coder, elem *exec.FullValue, eg yaml.MapItem) bool {
+	var got, want interface{}
+	switch c.Urn {
+	case "beam:coder:bytes:v1":
+		got = string(elem.Elm.([]byte))
+		switch egv := eg.Value.(type) {
+		case string:
+			want = egv
+		case []byte:
+			want = string(egv)
+		}
+	case "beam:coder:varint:v1":
+		got, want = elem.Elm.(int64), int64(eg.Value.(int))
+	case "beam:coder:double:v1":
+		got = elem.Elm.(float64)
+		switch v := eg.Value.(string); v {
+		case "NaN":
+			// Do the NaN comparison here since NaN by definition != NaN.
+			if math.IsNaN(got.(float64)) {
+				want, got = 1, 1
+			} else {
+				want = math.NaN()
+			}
+		case "-Infinity":
+			want = math.Inf(-1)
+		case "Infinity":
+			want = math.Inf(1)
+		default:
+			want, _ = strconv.ParseFloat(v, 64)
+		}
+
+	case "beam:coder:kv:v1":
+		v := eg.Value.(yaml.MapSlice)
+		pass := true
+		if !diff(c.Components[0], &exec.FullValue{Elm: elem.Elm}, v[0]) {
+			pass = false
+		}
+		if !diff(c.Components[1], &exec.FullValue{Elm: elem.Elm2}, v[1]) {
+			pass = false
+		}
+		return pass
+
+	case "beam:coder:iterable:v1":
+		pass := true
+		gotrv := reflect.ValueOf(elem.Elm)
+		wantrv := reflect.ValueOf(eg.Value)
+		if gotrv.Len() != wantrv.Len() {
+			log.Printf("Lengths don't match. got %v, want %v;  %v, %v", gotrv.Len(), wantrv.Len(), gotrv, wantrv)
+			return false
+		}
+		for i := 0; i < wantrv.Len(); i++ {
+			if !diff(c.Components[0],
+				&exec.FullValue{Elm: gotrv.Index(i).Interface()},
+				yaml.MapItem{Value: wantrv.Index(i).Interface()}) {
+				pass = false
+			}
+
+		}
+		return pass
+	case "beam:coder:interval_window:v1":
+		var a, b int
+		val := eg.Value
+		if is, ok := eg.Value.([]interface{}); ok {
+			val = is[0]
+		}
+		v := val.(yaml.MapSlice)
+
+		a = v[0].Value.(int)
+		b = v[1].Value.(int)
+		end := mtime.FromMilliseconds(int64(a))
+		start := end - mtime.Time(int64(b))
+		want = window.IntervalWindow{Start: start, End: end}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = elem.Elm
+		} else {
+			got = elem.Windows[0]
+		}
+
+	case "beam:coder:global_window:v1":
+		want = window.GlobalWindow{}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = window.GlobalWindow(elem.Elm.(struct{}))
+		} else {
+			got = elem.Windows[0]
+		}
+	case "beam:coder:windowed_value:v1", "beam:coder:param_windowed_value:v1":
+		// elem contains all the information, but we need to compare the element+timestamp
+		// separately from the windows, to avoid repeated expected value parsing logic.
+		pass := true
+		vs := eg.Value.(yaml.MapSlice)
+		if !diff(c.Components[0], elem, vs[0]) {
+			pass = false
+		}
+		if d := cmp.Diff(
+			mtime.FromMilliseconds(int64(vs[1].Value.(int))),
+			elem.Timestamp, cmpOpts...); d != "" {
+
+			pass = false
+		}
+		if !diff(c.Components[1], elem, vs[3]) {
+			pass = false
+		}
+		// TODO compare pane information.
+		return pass
+	case "beam:coder:row:v1":
+		fs := eg.Value.(yaml.MapSlice)
+		var rfs []reflect.StructField
+		// There are only 2 pointer examples, but they reuse field names,
+		// so we key off the proto hash to know which example we're handling.
+		ptrEg := strings.Contains(c.Payload, "51ace21c7393")
+		for _, rf := range fs {
+			name := rf.Key.(string)
+			t := nameToType[name]
+			if ptrEg {
+				t = reflect.PtrTo(t)
+			}
+			rfs = append(rfs, reflect.StructField{
+				Name: strings.ToUpper(name[:1]) + name[1:],
+				Type: t,
+				Tag:  reflect.StructTag(fmt.Sprintf("beam:\"%v\"", name)),
+			})
+		}
+		rv := reflect.New(reflect.StructOf(rfs)).Elem()
+		for i, rf := range fs {
+			setField(rv, i, rf.Value)
+		}
+
+		got, want = elem.Elm, rv.Interface()
+	default:
+		got, want = elem.Elm, eg.Value
+	}
+	if d := cmp.Diff(want, got, cmpOpts...); d != "" {
+		log.Printf("Decoding error: diff(-want,+got): %v\n", d)
+		return false
+	}
+	return true
+}
+
+// standard_coders.yaml uses the name for type indication, except for nullability.
+var nameToType = map[string]reflect.Type{
+	"str":     reflectx.String,
+	"i32":     reflectx.Int32,
+	"f64":     reflectx.Float64,
+	"arr":     reflect.SliceOf(reflectx.String),
+	"f_bool":  reflectx.Bool,
+	"f_bytes": reflect.PtrTo(reflectx.ByteSlice),
+	"f_map":   reflect.MapOf(reflectx.String, reflect.PtrTo(reflectx.Int64)),
+}
+
+func setField(rv reflect.Value, i int, v interface{}) {
+	if v == nil {
+		return
+	}
+	rf := rv.Field(i)
+	if rf.Kind() == reflect.Ptr {
+		// Ensure it's initialized.
+		rf.Set(reflect.New(rf.Type().Elem()))
+		rf = rf.Elem()
+	}
+	switch rf.Kind() {
+	case reflect.String:
+		rf.SetString(v.(string))
+	case reflect.Int32:
+		rf.SetInt(int64(v.(int)))
+	case reflect.Float64:
+		c, err := strconv.ParseFloat(v.(string), 64)
+		if err != nil {
+			panic(err)
+		}
+		rf.SetFloat(c)
+	case reflect.Slice:
+		if rf.Type() == reflectx.ByteSlice {
+			rf.Set(reflect.ValueOf([]byte(v.(string))))
+			break
+		}
+		// Value is a []interface{} with string values.

Review comment:
       Is this assumption made specifically based on the yaml file this is meant to be used with? Or is this based on something in the schema spec I'm forgetting?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)
+		if c.Custom.Name != "schema" {
+			return dec
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.
+		switch c.T.Type().Kind() {
+		case reflect.Slice:
+			return &lpDecoder{
+				dec: &iterableDecoder{
+					t:   c.Custom.Type,
+					dec: dec,
+				},
+			}
+		case reflect.Array:
+			return &lpDecoder{
+				dec: &arrayDecoder{

Review comment:
       Just to confirm, using an `arrayDecoder` without an `arrayEncoder` specifically works because the format that `iterableEncoder` encodes to is also decodable as an array, right? That might be worth mentioning as a comment, because I was wondering why you can decode arrays with an `arrayDecoder` if they were encoded with an `iterableEncoder`.

##########
File path: sdks/go/pkg/beam/core/runtime/graphx/coder.go
##########
@@ -249,31 +256,35 @@ func (b *CoderUnmarshaller) makeCoder(c *pipepb.Coder) (*coder.Coder, error) {
 			return nil, err
 		}
 
-		// No payload means this coder was length prefixed by the runner
-		// but is likely self describing - AKA a beam coder.
-		if len(sub.GetSpec().GetPayload()) == 0 {
-			return b.makeCoder(sub)
-		}
 		// TODO(lostluck) 2018/10/17: Make this strict again, once dataflow can use
 		// the portable pipeline model directly (BEAM-2885)
-		if sub.GetSpec().GetUrn() != "" && sub.GetSpec().GetUrn() != urnCustomCoder {
-			// TODO(herohde) 11/17/2017: revisit this restriction
-			return nil, errors.Errorf("could not unmarshal length prefix coder from %v, want a custom coder as a sub component but got %v", c, sub)
-		}
-
-		var ref v1pb.CustomCoder
-		if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
-			return nil, err
-		}
-		custom, err := decodeCustomCoder(&ref)
-		if err != nil {
-			return nil, err
+		switch u := sub.GetSpec().GetUrn(); u {
+		case "", urnCustomCoder:
+			var ref v1pb.CustomCoder
+			if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
+				return nil, err
+			}
+			custom, err := decodeCustomCoder(&ref)
+			if err != nil {
+				return nil, err
+			}
+			custom.ID = components[0]
+			t := typex.New(custom.Type)
+			cc := &coder.Coder{Kind: coder.Custom, T: t, Custom: custom}
+			fmt.Println("decoded customcoder", cc)

Review comment:
       Looks like a debug Println left in.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)

Review comment:
       Is it necessary to cast n to an int here? Isn't it already an int from line 638?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {
+		err := c.dec.DecodeTo(r, &e)
+		if err != nil {
+			return reflect.Value{}, err
+		}
+		if e.Elm != nil {
+			rv.Index(i).Set(reflect.ValueOf(e.Elm))
+		} else {
+			rv.Index(i).Set(reflect.ValueOf(e.Windows[0]))

Review comment:
       I don't get why it's setting the value of the element to `e.Windows[0]`. Is it a way to pass in nil?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {

Review comment:
       Likewise, is it necessary to cast n here too?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -81,24 +82,82 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder {
 		return &stringEncoder{}
 
 	case coder.Custom:
-		return &customEncoder{
+		enc := &customEncoder{
 			t:   c.Custom.Type,
 			enc: makeEncoder(c.Custom.Enc.Fn),
 		}
+		if c.Custom.Name != "schema" {
+			return enc
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.

Review comment:
       I don't really understand this comment, or how it relates to the check below.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)

Review comment:
       Looks like another debug println.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -534,26 +1038,34 @@ func (*intervalWindowEncoder) EncodeSingle(elm typex.Window, w io.Writer) error
 
 type intervalWindowDecoder struct{}
 
-func (*intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
+func (d *intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
 	// Encoding: upper bound and duration
 
 	n, err := coder.DecodeInt32(r) // #windows
 
 	ret := make([]typex.Window, n, n)
 	for i := int32(0); i < n; i++ {
-		end, err := coder.DecodeEventTime(r)
-		if err != nil {
-			return nil, err
-		}
-		duration, err := coder.DecodeVarUint64(r)
+		w, err := d.DecodeSingle(r)
 		if err != nil {
 			return nil, err
 		}
-		ret[i] = window.IntervalWindow{Start: mtime.FromMilliseconds(end.Milliseconds() - int64(duration)), End: end}
+		ret[i] = w
 	}
 	return ret, err
 }
 
+func (*intervalWindowDecoder) DecodeSingle(r io.Reader) (typex.Window, error) {
+	end, err := coder.DecodeEventTime(r)
+	if err != nil {
+		return nil, err
+	}
+	duration, err := coder.DecodeVarInt(r)

Review comment:
       What's the reason to switch from `DecodeVarUint64` to `DecodeVarInt` here?

##########
File path: sdks/go/test/regression/coders/fromyaml/fromyaml.go
##########
@@ -0,0 +1,415 @@
+// 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.
+
+// fromyaml generates a resource file from the standard_coders.yaml
+// file for use in these coder regression tests.
+//
+// It expects to be run in it's test directory, or via it's go test.
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"math"
+	"reflect"
+	"runtime/debug"
+	"strconv"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/google/go-cmp/cmp"
+	"golang.org/x/text/encoding/charmap"
+	yaml "gopkg.in/yaml.v2"
+)
+
+var unimplementedCoders = map[string]bool{
+	"beam:coder:param_windowed_value:v1": true,
+	"beam:coder:timer:v1":                true,
+}
+
+// Coder is a representation a serialized beam coder.
+type Coder struct {
+	Urn              string  `yaml:"urn,omitempty"`
+	Payload          string  `yaml:"payload,omitempty"`
+	Components       []Coder `yaml:"components,omitempty"`
+	NonDeterministic bool    `yaml:"non_deterministic,omitempty"`
+}
+
+type logger interface {
+	Errorf(string, ...interface{})
+	Logf(string, ...interface{})
+}
+
+// Spec is a set of conditions that a coder must pass.
+type Spec struct {
+	Coder    Coder         `yaml:"coder,omitempty"`
+	Nested   *bool         `yaml:"nested,omitempty"`
+	Examples yaml.MapSlice `yaml:"examples,omitempty"`
+	Log      logger
+
+	id       int // for generating coder ids.
+	coderPBs map[string]*pipepb.Coder
+}
+
+func (s *Spec) nextID() string {
+	ret := fmt.Sprintf("%d", s.id)
+	s.id++
+	return ret
+}
+
+func (s *Spec) testStandardCoder() (err error) {
+	if unimplementedCoders[s.Coder.Urn] {
+		log.Printf("skipping unimplemented coder urn: %v", s.Coder.Urn)
+		return nil
+	}
+	// Construct the coder proto equivalents.
+
+	// Only nested tests need to be run, since nestedness is a pre-portability
+	// concept.
+	// For legacy Java reasons, the row coder examples are all marked nested: false
+	// so we need to check that before skipping unnested tests.
+	if s.Coder.Urn != "beam:coder:row:v1" && s.Nested != nil && !*s.Nested {
+		log.Printf("skipping unnested coder spec: %v\n", s.Coder)
+		return nil
+	}
+
+	s.coderPBs = make(map[string]*pipepb.Coder)
+	id := s.parseCoder(s.Coder)
+	b := graphx.NewCoderUnmarshaller(s.coderPBs)
+	underTest, err := b.Coder(id)
+	if err != nil {
+		return fmt.Errorf("unable to create coder: %v", err)
+	}
+
+	defer func() {
+		if e := recover(); e != nil {
+			err = fmt.Errorf("panicked on coder %v || %v:\n\t%v :\n%s", underTest, s.Coder, e, debug.Stack())
+		}
+	}()
+
+	var decFails, encFails int
+	for _, eg := range s.Examples {
+
+		// Test Decoding
+		// Ideally we'd use the beam package coders, but KVs make that complicated.
+		// This can be cleaned up once a type parametered beam.KV type exists.
+		dec := exec.MakeElementDecoder(underTest)
+		encoded := eg.Key.(string)
+		var elem exec.FullValue
+
+		// What I would have expected.
+		//		r := charmap.ISO8859_1.NewDecoder().Reader(strings.NewReader(encoded))
+		recoded, err := charmap.ISO8859_1.NewEncoder().String(encoded)
+		if err != nil {
+			return err
+		}
+		r := strings.NewReader(recoded)
+		if err := dec.DecodeTo(r, &elem); err != nil {
+			return fmt.Errorf("err decoding %q: %v", encoded, err)
+		}
+		if !diff(s.Coder, &elem, eg) {
+			decFails++
+			continue
+		}
+
+		// Test Encoding
+		if s.Coder.NonDeterministic {
+			// Skip verifying nondeterministic encodings.
+			continue
+		}
+		enc := exec.MakeElementEncoder(underTest)
+		var out bytes.Buffer
+		if err := enc.Encode(&elem, &out); err != nil {
+			return err
+		}
+		if d := cmp.Diff(recoded, string(out.Bytes())); d != "" {
+			log.Printf("Encoding error: diff(-want,+got): %v\n", d)
+		}
+	}
+	if decFails+encFails > 0 {
+		return fmt.Errorf("failed to decode %v times, and encode %v times", decFails, encFails)
+	}
+
+	return nil
+}
+
+var cmpOpts = []cmp.Option{
+	cmp.Transformer("bytes2string", func(in []byte) (out string) {
+		return string(in)
+	}),
+}
+
+func diff(c Coder, elem *exec.FullValue, eg yaml.MapItem) bool {
+	var got, want interface{}
+	switch c.Urn {
+	case "beam:coder:bytes:v1":
+		got = string(elem.Elm.([]byte))
+		switch egv := eg.Value.(type) {
+		case string:
+			want = egv
+		case []byte:
+			want = string(egv)
+		}
+	case "beam:coder:varint:v1":
+		got, want = elem.Elm.(int64), int64(eg.Value.(int))
+	case "beam:coder:double:v1":
+		got = elem.Elm.(float64)
+		switch v := eg.Value.(string); v {
+		case "NaN":
+			// Do the NaN comparison here since NaN by definition != NaN.
+			if math.IsNaN(got.(float64)) {
+				want, got = 1, 1
+			} else {
+				want = math.NaN()
+			}
+		case "-Infinity":
+			want = math.Inf(-1)
+		case "Infinity":
+			want = math.Inf(1)
+		default:
+			want, _ = strconv.ParseFloat(v, 64)
+		}
+
+	case "beam:coder:kv:v1":
+		v := eg.Value.(yaml.MapSlice)
+		pass := true
+		if !diff(c.Components[0], &exec.FullValue{Elm: elem.Elm}, v[0]) {
+			pass = false
+		}
+		if !diff(c.Components[1], &exec.FullValue{Elm: elem.Elm2}, v[1]) {
+			pass = false
+		}
+		return pass
+
+	case "beam:coder:iterable:v1":
+		pass := true
+		gotrv := reflect.ValueOf(elem.Elm)
+		wantrv := reflect.ValueOf(eg.Value)
+		if gotrv.Len() != wantrv.Len() {
+			log.Printf("Lengths don't match. got %v, want %v;  %v, %v", gotrv.Len(), wantrv.Len(), gotrv, wantrv)
+			return false
+		}
+		for i := 0; i < wantrv.Len(); i++ {
+			if !diff(c.Components[0],
+				&exec.FullValue{Elm: gotrv.Index(i).Interface()},
+				yaml.MapItem{Value: wantrv.Index(i).Interface()}) {
+				pass = false
+			}
+
+		}
+		return pass
+	case "beam:coder:interval_window:v1":
+		var a, b int
+		val := eg.Value
+		if is, ok := eg.Value.([]interface{}); ok {
+			val = is[0]
+		}
+		v := val.(yaml.MapSlice)
+
+		a = v[0].Value.(int)
+		b = v[1].Value.(int)
+		end := mtime.FromMilliseconds(int64(a))
+		start := end - mtime.Time(int64(b))
+		want = window.IntervalWindow{Start: start, End: end}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = elem.Elm
+		} else {
+			got = elem.Windows[0]
+		}
+
+	case "beam:coder:global_window:v1":
+		want = window.GlobalWindow{}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = window.GlobalWindow(elem.Elm.(struct{}))
+		} else {
+			got = elem.Windows[0]
+		}
+	case "beam:coder:windowed_value:v1", "beam:coder:param_windowed_value:v1":
+		// elem contains all the information, but we need to compare the element+timestamp
+		// separately from the windows, to avoid repeated expected value parsing logic.
+		pass := true
+		vs := eg.Value.(yaml.MapSlice)
+		if !diff(c.Components[0], elem, vs[0]) {
+			pass = false
+		}
+		if d := cmp.Diff(
+			mtime.FromMilliseconds(int64(vs[1].Value.(int))),
+			elem.Timestamp, cmpOpts...); d != "" {
+
+			pass = false
+		}
+		if !diff(c.Components[1], elem, vs[3]) {
+			pass = false
+		}
+		// TODO compare pane information.
+		return pass
+	case "beam:coder:row:v1":
+		fs := eg.Value.(yaml.MapSlice)
+		var rfs []reflect.StructField
+		// There are only 2 pointer examples, but they reuse field names,
+		// so we key off the proto hash to know which example we're handling.
+		ptrEg := strings.Contains(c.Payload, "51ace21c7393")
+		for _, rf := range fs {
+			name := rf.Key.(string)
+			t := nameToType[name]
+			if ptrEg {
+				t = reflect.PtrTo(t)
+			}
+			rfs = append(rfs, reflect.StructField{
+				Name: strings.ToUpper(name[:1]) + name[1:],
+				Type: t,
+				Tag:  reflect.StructTag(fmt.Sprintf("beam:\"%v\"", name)),
+			})
+		}
+		rv := reflect.New(reflect.StructOf(rfs)).Elem()
+		for i, rf := range fs {
+			setField(rv, i, rf.Value)
+		}
+
+		got, want = elem.Elm, rv.Interface()
+	default:
+		got, want = elem.Elm, eg.Value
+	}
+	if d := cmp.Diff(want, got, cmpOpts...); d != "" {
+		log.Printf("Decoding error: diff(-want,+got): %v\n", d)
+		return false
+	}
+	return true
+}
+
+// standard_coders.yaml uses the name for type indication, except for nullability.
+var nameToType = map[string]reflect.Type{
+	"str":     reflectx.String,
+	"i32":     reflectx.Int32,
+	"f64":     reflectx.Float64,
+	"arr":     reflect.SliceOf(reflectx.String),
+	"f_bool":  reflectx.Bool,
+	"f_bytes": reflect.PtrTo(reflectx.ByteSlice),
+	"f_map":   reflect.MapOf(reflectx.String, reflect.PtrTo(reflectx.Int64)),
+}
+
+func setField(rv reflect.Value, i int, v interface{}) {
+	if v == nil {
+		return
+	}
+	rf := rv.Field(i)
+	if rf.Kind() == reflect.Ptr {
+		// Ensure it's initialized.
+		rf.Set(reflect.New(rf.Type().Elem()))
+		rf = rf.Elem()
+	}
+	switch rf.Kind() {
+	case reflect.String:
+		rf.SetString(v.(string))
+	case reflect.Int32:
+		rf.SetInt(int64(v.(int)))
+	case reflect.Float64:
+		c, err := strconv.ParseFloat(v.(string), 64)
+		if err != nil {
+			panic(err)
+		}
+		rf.SetFloat(c)
+	case reflect.Slice:
+		if rf.Type() == reflectx.ByteSlice {
+			rf.Set(reflect.ValueOf([]byte(v.(string))))
+			break
+		}
+		// Value is a []interface{} with string values.

Review comment:
       Is this assumption made specifically based on the yaml file this is meant to be used with? Or is this based on something in the schema spec I'm forgetting?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)
+		if c.Custom.Name != "schema" {
+			return dec
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.
+		switch c.T.Type().Kind() {
+		case reflect.Slice:
+			return &lpDecoder{
+				dec: &iterableDecoder{
+					t:   c.Custom.Type,
+					dec: dec,
+				},
+			}
+		case reflect.Array:
+			return &lpDecoder{
+				dec: &arrayDecoder{

Review comment:
       Just to confirm, using an `arrayDecoder` without an `arrayEncoder` specifically works because the format that `iterableEncoder` encodes to is also decodable as an array, right? That might be worth mentioning as a comment, because I was wondering why you can decode arrays with an `arrayDecoder` if they were encoded with an `iterableEncoder`.

##########
File path: sdks/go/pkg/beam/core/runtime/graphx/coder.go
##########
@@ -249,31 +256,35 @@ func (b *CoderUnmarshaller) makeCoder(c *pipepb.Coder) (*coder.Coder, error) {
 			return nil, err
 		}
 
-		// No payload means this coder was length prefixed by the runner
-		// but is likely self describing - AKA a beam coder.
-		if len(sub.GetSpec().GetPayload()) == 0 {
-			return b.makeCoder(sub)
-		}
 		// TODO(lostluck) 2018/10/17: Make this strict again, once dataflow can use
 		// the portable pipeline model directly (BEAM-2885)
-		if sub.GetSpec().GetUrn() != "" && sub.GetSpec().GetUrn() != urnCustomCoder {
-			// TODO(herohde) 11/17/2017: revisit this restriction
-			return nil, errors.Errorf("could not unmarshal length prefix coder from %v, want a custom coder as a sub component but got %v", c, sub)
-		}
-
-		var ref v1pb.CustomCoder
-		if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
-			return nil, err
-		}
-		custom, err := decodeCustomCoder(&ref)
-		if err != nil {
-			return nil, err
+		switch u := sub.GetSpec().GetUrn(); u {
+		case "", urnCustomCoder:
+			var ref v1pb.CustomCoder
+			if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
+				return nil, err
+			}
+			custom, err := decodeCustomCoder(&ref)
+			if err != nil {
+				return nil, err
+			}
+			custom.ID = components[0]
+			t := typex.New(custom.Type)
+			cc := &coder.Coder{Kind: coder.Custom, T: t, Custom: custom}
+			fmt.Println("decoded customcoder", cc)

Review comment:
       Looks like a debug Println left in.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)

Review comment:
       Is it necessary to cast n to an int here? Isn't it already an int from line 638?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {
+		err := c.dec.DecodeTo(r, &e)
+		if err != nil {
+			return reflect.Value{}, err
+		}
+		if e.Elm != nil {
+			rv.Index(i).Set(reflect.ValueOf(e.Elm))
+		} else {
+			rv.Index(i).Set(reflect.ValueOf(e.Windows[0]))

Review comment:
       I don't get why it's setting the value of the element to `e.Windows[0]`. Is it a way to pass in nil?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {

Review comment:
       Likewise, is it necessary to cast n here too?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/c8991ba9b65af2f547bad68a0ac327ce720eb00b?el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #12588      +/-   ##
   ==========================================
   - Coverage   82.30%   82.29%   -0.01%     
   ==========================================
     Files         451      451              
     Lines       53818    53818              
   ==========================================
   - Hits        44294    44289       -5     
   - Misses       9524     9529       +5     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/coders/row\_coder.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vY29kZXJzL3Jvd19jb2Rlci5weQ==) | `93.66% <0.00%> (-0.71%)` | :arrow_down: |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `88.98% <0.00%> (-0.36%)` | :arrow_down: |
   | [...hon/apache\_beam/runners/worker/bundle\_processor.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvYnVuZGxlX3Byb2Nlc3Nvci5weQ==) | `94.18% <0.00%> (-0.27%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [c8991ba...4d5eaa2](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/0f9d0020aaaa338c1916cba55b7877f4ae897e46?el=desc) will **decrease** coverage by `41.93%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff             @@
   ##           master   #12588       +/-   ##
   ===========================================
   - Coverage   82.32%   40.38%   -41.94%     
   ===========================================
     Files         451      451               
     Lines       53818    53721       -97     
   ===========================================
   - Hits        44304    21696    -22608     
   - Misses       9514    32025    +22511     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...python/apache\_beam/examples/complete/distribopt.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZXhhbXBsZXMvY29tcGxldGUvZGlzdHJpYm9wdC5weQ==) | `0.00% <0.00%> (-98.59%)` | :arrow_down: |
   | [...dks/python/apache\_beam/transforms/create\_source.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdHJhbnNmb3Jtcy9jcmVhdGVfc291cmNlLnB5) | `0.00% <0.00%> (-98.19%)` | :arrow_down: |
   | [...on/apache\_beam/runners/direct/helper\_transforms.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvaGVscGVyX3RyYW5zZm9ybXMucHk=) | `0.00% <0.00%> (-98.15%)` | :arrow_down: |
   | [...e\_beam/runners/interactive/testing/mock\_ipython.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS90ZXN0aW5nL21vY2tfaXB5dGhvbi5weQ==) | `7.14% <0.00%> (-92.86%)` | :arrow_down: |
   | [.../examples/snippets/transforms/elementwise/pardo.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZXhhbXBsZXMvc25pcHBldHMvdHJhbnNmb3Jtcy9lbGVtZW50d2lzZS9wYXJkby5weQ==) | `11.36% <0.00%> (-88.64%)` | :arrow_down: |
   | [sdks/python/apache\_beam/typehints/opcodes.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdHlwZWhpbnRzL29wY29kZXMucHk=) | `0.00% <0.00%> (-87.66%)` | :arrow_down: |
   | [...s/snippets/transforms/aggregation/combineperkey.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZXhhbXBsZXMvc25pcHBldHMvdHJhbnNmb3Jtcy9hZ2dyZWdhdGlvbi9jb21iaW5lcGVya2V5LnB5) | `11.95% <0.00%> (-86.96%)` | :arrow_down: |
   | [...xamples/snippets/transforms/elementwise/flatmap.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZXhhbXBsZXMvc25pcHBldHMvdHJhbnNmb3Jtcy9lbGVtZW50d2lzZS9mbGF0bWFwLnB5) | `14.28% <0.00%> (-85.72%)` | :arrow_down: |
   | [...mples/snippets/transforms/elementwise/partition.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZXhhbXBsZXMvc25pcHBldHMvdHJhbnNmb3Jtcy9lbGVtZW50d2lzZS9wYXJ0aXRpb24ucHk=) | `11.90% <0.00%> (-85.72%)` | :arrow_down: |
   | [...ache\_beam/runners/interactive/pipeline\_fragment.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9waXBlbGluZV9mcmFnbWVudC5weQ==) | `13.39% <0.00%> (-84.83%)` | :arrow_down: |
   | ... and [271 more](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [c8991ba...4f42690](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-693636103


   No major changes. I've hacked in to ignore tests with logical type for now to get this in, as that's something I want to handle in a different PR. Will merge after the test passes.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] youngoli commented on a change in pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #12588:
URL: https://github.com/apache/beam/pull/12588#discussion_r486767704



##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -81,24 +82,82 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder {
 		return &stringEncoder{}
 
 	case coder.Custom:
-		return &customEncoder{
+		enc := &customEncoder{
 			t:   c.Custom.Type,
 			enc: makeEncoder(c.Custom.Enc.Fn),
 		}
+		if c.Custom.Name != "schema" {
+			return enc
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.

Review comment:
       I don't really understand this comment, or how it relates to the check below.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)

Review comment:
       Looks like another debug println.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -534,26 +1038,34 @@ func (*intervalWindowEncoder) EncodeSingle(elm typex.Window, w io.Writer) error
 
 type intervalWindowDecoder struct{}
 
-func (*intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
+func (d *intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
 	// Encoding: upper bound and duration
 
 	n, err := coder.DecodeInt32(r) // #windows
 
 	ret := make([]typex.Window, n, n)
 	for i := int32(0); i < n; i++ {
-		end, err := coder.DecodeEventTime(r)
-		if err != nil {
-			return nil, err
-		}
-		duration, err := coder.DecodeVarUint64(r)
+		w, err := d.DecodeSingle(r)
 		if err != nil {
 			return nil, err
 		}
-		ret[i] = window.IntervalWindow{Start: mtime.FromMilliseconds(end.Milliseconds() - int64(duration)), End: end}
+		ret[i] = w
 	}
 	return ret, err
 }
 
+func (*intervalWindowDecoder) DecodeSingle(r io.Reader) (typex.Window, error) {
+	end, err := coder.DecodeEventTime(r)
+	if err != nil {
+		return nil, err
+	}
+	duration, err := coder.DecodeVarInt(r)

Review comment:
       What's the reason to switch from `DecodeVarUint64` to `DecodeVarInt` here?

##########
File path: sdks/go/test/regression/coders/fromyaml/fromyaml.go
##########
@@ -0,0 +1,415 @@
+// 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.
+
+// fromyaml generates a resource file from the standard_coders.yaml
+// file for use in these coder regression tests.
+//
+// It expects to be run in it's test directory, or via it's go test.
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"math"
+	"reflect"
+	"runtime/debug"
+	"strconv"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/google/go-cmp/cmp"
+	"golang.org/x/text/encoding/charmap"
+	yaml "gopkg.in/yaml.v2"
+)
+
+var unimplementedCoders = map[string]bool{
+	"beam:coder:param_windowed_value:v1": true,
+	"beam:coder:timer:v1":                true,
+}
+
+// Coder is a representation a serialized beam coder.
+type Coder struct {
+	Urn              string  `yaml:"urn,omitempty"`
+	Payload          string  `yaml:"payload,omitempty"`
+	Components       []Coder `yaml:"components,omitempty"`
+	NonDeterministic bool    `yaml:"non_deterministic,omitempty"`
+}
+
+type logger interface {
+	Errorf(string, ...interface{})
+	Logf(string, ...interface{})
+}
+
+// Spec is a set of conditions that a coder must pass.
+type Spec struct {
+	Coder    Coder         `yaml:"coder,omitempty"`
+	Nested   *bool         `yaml:"nested,omitempty"`
+	Examples yaml.MapSlice `yaml:"examples,omitempty"`
+	Log      logger
+
+	id       int // for generating coder ids.
+	coderPBs map[string]*pipepb.Coder
+}
+
+func (s *Spec) nextID() string {
+	ret := fmt.Sprintf("%d", s.id)
+	s.id++
+	return ret
+}
+
+func (s *Spec) testStandardCoder() (err error) {
+	if unimplementedCoders[s.Coder.Urn] {
+		log.Printf("skipping unimplemented coder urn: %v", s.Coder.Urn)
+		return nil
+	}
+	// Construct the coder proto equivalents.
+
+	// Only nested tests need to be run, since nestedness is a pre-portability
+	// concept.
+	// For legacy Java reasons, the row coder examples are all marked nested: false
+	// so we need to check that before skipping unnested tests.
+	if s.Coder.Urn != "beam:coder:row:v1" && s.Nested != nil && !*s.Nested {
+		log.Printf("skipping unnested coder spec: %v\n", s.Coder)
+		return nil
+	}
+
+	s.coderPBs = make(map[string]*pipepb.Coder)
+	id := s.parseCoder(s.Coder)
+	b := graphx.NewCoderUnmarshaller(s.coderPBs)
+	underTest, err := b.Coder(id)
+	if err != nil {
+		return fmt.Errorf("unable to create coder: %v", err)
+	}
+
+	defer func() {
+		if e := recover(); e != nil {
+			err = fmt.Errorf("panicked on coder %v || %v:\n\t%v :\n%s", underTest, s.Coder, e, debug.Stack())
+		}
+	}()
+
+	var decFails, encFails int
+	for _, eg := range s.Examples {
+
+		// Test Decoding
+		// Ideally we'd use the beam package coders, but KVs make that complicated.
+		// This can be cleaned up once a type parametered beam.KV type exists.
+		dec := exec.MakeElementDecoder(underTest)
+		encoded := eg.Key.(string)
+		var elem exec.FullValue
+
+		// What I would have expected.
+		//		r := charmap.ISO8859_1.NewDecoder().Reader(strings.NewReader(encoded))
+		recoded, err := charmap.ISO8859_1.NewEncoder().String(encoded)
+		if err != nil {
+			return err
+		}
+		r := strings.NewReader(recoded)
+		if err := dec.DecodeTo(r, &elem); err != nil {
+			return fmt.Errorf("err decoding %q: %v", encoded, err)
+		}
+		if !diff(s.Coder, &elem, eg) {
+			decFails++
+			continue
+		}
+
+		// Test Encoding
+		if s.Coder.NonDeterministic {
+			// Skip verifying nondeterministic encodings.
+			continue
+		}
+		enc := exec.MakeElementEncoder(underTest)
+		var out bytes.Buffer
+		if err := enc.Encode(&elem, &out); err != nil {
+			return err
+		}
+		if d := cmp.Diff(recoded, string(out.Bytes())); d != "" {
+			log.Printf("Encoding error: diff(-want,+got): %v\n", d)
+		}
+	}
+	if decFails+encFails > 0 {
+		return fmt.Errorf("failed to decode %v times, and encode %v times", decFails, encFails)
+	}
+
+	return nil
+}
+
+var cmpOpts = []cmp.Option{
+	cmp.Transformer("bytes2string", func(in []byte) (out string) {
+		return string(in)
+	}),
+}
+
+func diff(c Coder, elem *exec.FullValue, eg yaml.MapItem) bool {
+	var got, want interface{}
+	switch c.Urn {
+	case "beam:coder:bytes:v1":
+		got = string(elem.Elm.([]byte))
+		switch egv := eg.Value.(type) {
+		case string:
+			want = egv
+		case []byte:
+			want = string(egv)
+		}
+	case "beam:coder:varint:v1":
+		got, want = elem.Elm.(int64), int64(eg.Value.(int))
+	case "beam:coder:double:v1":
+		got = elem.Elm.(float64)
+		switch v := eg.Value.(string); v {
+		case "NaN":
+			// Do the NaN comparison here since NaN by definition != NaN.
+			if math.IsNaN(got.(float64)) {
+				want, got = 1, 1
+			} else {
+				want = math.NaN()
+			}
+		case "-Infinity":
+			want = math.Inf(-1)
+		case "Infinity":
+			want = math.Inf(1)
+		default:
+			want, _ = strconv.ParseFloat(v, 64)
+		}
+
+	case "beam:coder:kv:v1":
+		v := eg.Value.(yaml.MapSlice)
+		pass := true
+		if !diff(c.Components[0], &exec.FullValue{Elm: elem.Elm}, v[0]) {
+			pass = false
+		}
+		if !diff(c.Components[1], &exec.FullValue{Elm: elem.Elm2}, v[1]) {
+			pass = false
+		}
+		return pass
+
+	case "beam:coder:iterable:v1":
+		pass := true
+		gotrv := reflect.ValueOf(elem.Elm)
+		wantrv := reflect.ValueOf(eg.Value)
+		if gotrv.Len() != wantrv.Len() {
+			log.Printf("Lengths don't match. got %v, want %v;  %v, %v", gotrv.Len(), wantrv.Len(), gotrv, wantrv)
+			return false
+		}
+		for i := 0; i < wantrv.Len(); i++ {
+			if !diff(c.Components[0],
+				&exec.FullValue{Elm: gotrv.Index(i).Interface()},
+				yaml.MapItem{Value: wantrv.Index(i).Interface()}) {
+				pass = false
+			}
+
+		}
+		return pass
+	case "beam:coder:interval_window:v1":
+		var a, b int
+		val := eg.Value
+		if is, ok := eg.Value.([]interface{}); ok {
+			val = is[0]
+		}
+		v := val.(yaml.MapSlice)
+
+		a = v[0].Value.(int)
+		b = v[1].Value.(int)
+		end := mtime.FromMilliseconds(int64(a))
+		start := end - mtime.Time(int64(b))
+		want = window.IntervalWindow{Start: start, End: end}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = elem.Elm
+		} else {
+			got = elem.Windows[0]
+		}
+
+	case "beam:coder:global_window:v1":
+		want = window.GlobalWindow{}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = window.GlobalWindow(elem.Elm.(struct{}))
+		} else {
+			got = elem.Windows[0]
+		}
+	case "beam:coder:windowed_value:v1", "beam:coder:param_windowed_value:v1":
+		// elem contains all the information, but we need to compare the element+timestamp
+		// separately from the windows, to avoid repeated expected value parsing logic.
+		pass := true
+		vs := eg.Value.(yaml.MapSlice)
+		if !diff(c.Components[0], elem, vs[0]) {
+			pass = false
+		}
+		if d := cmp.Diff(
+			mtime.FromMilliseconds(int64(vs[1].Value.(int))),
+			elem.Timestamp, cmpOpts...); d != "" {
+
+			pass = false
+		}
+		if !diff(c.Components[1], elem, vs[3]) {
+			pass = false
+		}
+		// TODO compare pane information.
+		return pass
+	case "beam:coder:row:v1":
+		fs := eg.Value.(yaml.MapSlice)
+		var rfs []reflect.StructField
+		// There are only 2 pointer examples, but they reuse field names,
+		// so we key off the proto hash to know which example we're handling.
+		ptrEg := strings.Contains(c.Payload, "51ace21c7393")
+		for _, rf := range fs {
+			name := rf.Key.(string)
+			t := nameToType[name]
+			if ptrEg {
+				t = reflect.PtrTo(t)
+			}
+			rfs = append(rfs, reflect.StructField{
+				Name: strings.ToUpper(name[:1]) + name[1:],
+				Type: t,
+				Tag:  reflect.StructTag(fmt.Sprintf("beam:\"%v\"", name)),
+			})
+		}
+		rv := reflect.New(reflect.StructOf(rfs)).Elem()
+		for i, rf := range fs {
+			setField(rv, i, rf.Value)
+		}
+
+		got, want = elem.Elm, rv.Interface()
+	default:
+		got, want = elem.Elm, eg.Value
+	}
+	if d := cmp.Diff(want, got, cmpOpts...); d != "" {
+		log.Printf("Decoding error: diff(-want,+got): %v\n", d)
+		return false
+	}
+	return true
+}
+
+// standard_coders.yaml uses the name for type indication, except for nullability.
+var nameToType = map[string]reflect.Type{
+	"str":     reflectx.String,
+	"i32":     reflectx.Int32,
+	"f64":     reflectx.Float64,
+	"arr":     reflect.SliceOf(reflectx.String),
+	"f_bool":  reflectx.Bool,
+	"f_bytes": reflect.PtrTo(reflectx.ByteSlice),
+	"f_map":   reflect.MapOf(reflectx.String, reflect.PtrTo(reflectx.Int64)),
+}
+
+func setField(rv reflect.Value, i int, v interface{}) {
+	if v == nil {
+		return
+	}
+	rf := rv.Field(i)
+	if rf.Kind() == reflect.Ptr {
+		// Ensure it's initialized.
+		rf.Set(reflect.New(rf.Type().Elem()))
+		rf = rf.Elem()
+	}
+	switch rf.Kind() {
+	case reflect.String:
+		rf.SetString(v.(string))
+	case reflect.Int32:
+		rf.SetInt(int64(v.(int)))
+	case reflect.Float64:
+		c, err := strconv.ParseFloat(v.(string), 64)
+		if err != nil {
+			panic(err)
+		}
+		rf.SetFloat(c)
+	case reflect.Slice:
+		if rf.Type() == reflectx.ByteSlice {
+			rf.Set(reflect.ValueOf([]byte(v.(string))))
+			break
+		}
+		// Value is a []interface{} with string values.

Review comment:
       Is this assumption made specifically based on the yaml file this is meant to be used with? Or is this based on something in the schema spec I'm forgetting?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)
+		if c.Custom.Name != "schema" {
+			return dec
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.
+		switch c.T.Type().Kind() {
+		case reflect.Slice:
+			return &lpDecoder{
+				dec: &iterableDecoder{
+					t:   c.Custom.Type,
+					dec: dec,
+				},
+			}
+		case reflect.Array:
+			return &lpDecoder{
+				dec: &arrayDecoder{

Review comment:
       Just to confirm, using an `arrayDecoder` without an `arrayEncoder` specifically works because the format that `iterableEncoder` encodes to is also decodable as an array, right? That might be worth mentioning as a comment, because I was wondering why you can decode arrays with an `arrayDecoder` if they were encoded with an `iterableEncoder`.

##########
File path: sdks/go/pkg/beam/core/runtime/graphx/coder.go
##########
@@ -249,31 +256,35 @@ func (b *CoderUnmarshaller) makeCoder(c *pipepb.Coder) (*coder.Coder, error) {
 			return nil, err
 		}
 
-		// No payload means this coder was length prefixed by the runner
-		// but is likely self describing - AKA a beam coder.
-		if len(sub.GetSpec().GetPayload()) == 0 {
-			return b.makeCoder(sub)
-		}
 		// TODO(lostluck) 2018/10/17: Make this strict again, once dataflow can use
 		// the portable pipeline model directly (BEAM-2885)
-		if sub.GetSpec().GetUrn() != "" && sub.GetSpec().GetUrn() != urnCustomCoder {
-			// TODO(herohde) 11/17/2017: revisit this restriction
-			return nil, errors.Errorf("could not unmarshal length prefix coder from %v, want a custom coder as a sub component but got %v", c, sub)
-		}
-
-		var ref v1pb.CustomCoder
-		if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
-			return nil, err
-		}
-		custom, err := decodeCustomCoder(&ref)
-		if err != nil {
-			return nil, err
+		switch u := sub.GetSpec().GetUrn(); u {
+		case "", urnCustomCoder:
+			var ref v1pb.CustomCoder
+			if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
+				return nil, err
+			}
+			custom, err := decodeCustomCoder(&ref)
+			if err != nil {
+				return nil, err
+			}
+			custom.ID = components[0]
+			t := typex.New(custom.Type)
+			cc := &coder.Coder{Kind: coder.Custom, T: t, Custom: custom}
+			fmt.Println("decoded customcoder", cc)

Review comment:
       Looks like a debug Println left in.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)

Review comment:
       Is it necessary to cast n to an int here? Isn't it already an int from line 638?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {
+		err := c.dec.DecodeTo(r, &e)
+		if err != nil {
+			return reflect.Value{}, err
+		}
+		if e.Elm != nil {
+			rv.Index(i).Set(reflect.ValueOf(e.Elm))
+		} else {
+			rv.Index(i).Set(reflect.ValueOf(e.Windows[0]))

Review comment:
       I don't get why it's setting the value of the element to `e.Windows[0]`. Is it a way to pass in nil?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {

Review comment:
       Likewise, is it necessary to cast n here too?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] youngoli commented on a change in pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #12588:
URL: https://github.com/apache/beam/pull/12588#discussion_r486767704



##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -81,24 +82,82 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder {
 		return &stringEncoder{}
 
 	case coder.Custom:
-		return &customEncoder{
+		enc := &customEncoder{
 			t:   c.Custom.Type,
 			enc: makeEncoder(c.Custom.Enc.Fn),
 		}
+		if c.Custom.Name != "schema" {
+			return enc
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.

Review comment:
       I don't really understand this comment, or how it relates to the check below.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)

Review comment:
       Looks like another debug println.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -534,26 +1038,34 @@ func (*intervalWindowEncoder) EncodeSingle(elm typex.Window, w io.Writer) error
 
 type intervalWindowDecoder struct{}
 
-func (*intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
+func (d *intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
 	// Encoding: upper bound and duration
 
 	n, err := coder.DecodeInt32(r) // #windows
 
 	ret := make([]typex.Window, n, n)
 	for i := int32(0); i < n; i++ {
-		end, err := coder.DecodeEventTime(r)
-		if err != nil {
-			return nil, err
-		}
-		duration, err := coder.DecodeVarUint64(r)
+		w, err := d.DecodeSingle(r)
 		if err != nil {
 			return nil, err
 		}
-		ret[i] = window.IntervalWindow{Start: mtime.FromMilliseconds(end.Milliseconds() - int64(duration)), End: end}
+		ret[i] = w
 	}
 	return ret, err
 }
 
+func (*intervalWindowDecoder) DecodeSingle(r io.Reader) (typex.Window, error) {
+	end, err := coder.DecodeEventTime(r)
+	if err != nil {
+		return nil, err
+	}
+	duration, err := coder.DecodeVarInt(r)

Review comment:
       What's the reason to switch from `DecodeVarUint64` to `DecodeVarInt` here?

##########
File path: sdks/go/test/regression/coders/fromyaml/fromyaml.go
##########
@@ -0,0 +1,415 @@
+// 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.
+
+// fromyaml generates a resource file from the standard_coders.yaml
+// file for use in these coder regression tests.
+//
+// It expects to be run in it's test directory, or via it's go test.
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"math"
+	"reflect"
+	"runtime/debug"
+	"strconv"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/google/go-cmp/cmp"
+	"golang.org/x/text/encoding/charmap"
+	yaml "gopkg.in/yaml.v2"
+)
+
+var unimplementedCoders = map[string]bool{
+	"beam:coder:param_windowed_value:v1": true,
+	"beam:coder:timer:v1":                true,
+}
+
+// Coder is a representation a serialized beam coder.
+type Coder struct {
+	Urn              string  `yaml:"urn,omitempty"`
+	Payload          string  `yaml:"payload,omitempty"`
+	Components       []Coder `yaml:"components,omitempty"`
+	NonDeterministic bool    `yaml:"non_deterministic,omitempty"`
+}
+
+type logger interface {
+	Errorf(string, ...interface{})
+	Logf(string, ...interface{})
+}
+
+// Spec is a set of conditions that a coder must pass.
+type Spec struct {
+	Coder    Coder         `yaml:"coder,omitempty"`
+	Nested   *bool         `yaml:"nested,omitempty"`
+	Examples yaml.MapSlice `yaml:"examples,omitempty"`
+	Log      logger
+
+	id       int // for generating coder ids.
+	coderPBs map[string]*pipepb.Coder
+}
+
+func (s *Spec) nextID() string {
+	ret := fmt.Sprintf("%d", s.id)
+	s.id++
+	return ret
+}
+
+func (s *Spec) testStandardCoder() (err error) {
+	if unimplementedCoders[s.Coder.Urn] {
+		log.Printf("skipping unimplemented coder urn: %v", s.Coder.Urn)
+		return nil
+	}
+	// Construct the coder proto equivalents.
+
+	// Only nested tests need to be run, since nestedness is a pre-portability
+	// concept.
+	// For legacy Java reasons, the row coder examples are all marked nested: false
+	// so we need to check that before skipping unnested tests.
+	if s.Coder.Urn != "beam:coder:row:v1" && s.Nested != nil && !*s.Nested {
+		log.Printf("skipping unnested coder spec: %v\n", s.Coder)
+		return nil
+	}
+
+	s.coderPBs = make(map[string]*pipepb.Coder)
+	id := s.parseCoder(s.Coder)
+	b := graphx.NewCoderUnmarshaller(s.coderPBs)
+	underTest, err := b.Coder(id)
+	if err != nil {
+		return fmt.Errorf("unable to create coder: %v", err)
+	}
+
+	defer func() {
+		if e := recover(); e != nil {
+			err = fmt.Errorf("panicked on coder %v || %v:\n\t%v :\n%s", underTest, s.Coder, e, debug.Stack())
+		}
+	}()
+
+	var decFails, encFails int
+	for _, eg := range s.Examples {
+
+		// Test Decoding
+		// Ideally we'd use the beam package coders, but KVs make that complicated.
+		// This can be cleaned up once a type parametered beam.KV type exists.
+		dec := exec.MakeElementDecoder(underTest)
+		encoded := eg.Key.(string)
+		var elem exec.FullValue
+
+		// What I would have expected.
+		//		r := charmap.ISO8859_1.NewDecoder().Reader(strings.NewReader(encoded))
+		recoded, err := charmap.ISO8859_1.NewEncoder().String(encoded)
+		if err != nil {
+			return err
+		}
+		r := strings.NewReader(recoded)
+		if err := dec.DecodeTo(r, &elem); err != nil {
+			return fmt.Errorf("err decoding %q: %v", encoded, err)
+		}
+		if !diff(s.Coder, &elem, eg) {
+			decFails++
+			continue
+		}
+
+		// Test Encoding
+		if s.Coder.NonDeterministic {
+			// Skip verifying nondeterministic encodings.
+			continue
+		}
+		enc := exec.MakeElementEncoder(underTest)
+		var out bytes.Buffer
+		if err := enc.Encode(&elem, &out); err != nil {
+			return err
+		}
+		if d := cmp.Diff(recoded, string(out.Bytes())); d != "" {
+			log.Printf("Encoding error: diff(-want,+got): %v\n", d)
+		}
+	}
+	if decFails+encFails > 0 {
+		return fmt.Errorf("failed to decode %v times, and encode %v times", decFails, encFails)
+	}
+
+	return nil
+}
+
+var cmpOpts = []cmp.Option{
+	cmp.Transformer("bytes2string", func(in []byte) (out string) {
+		return string(in)
+	}),
+}
+
+func diff(c Coder, elem *exec.FullValue, eg yaml.MapItem) bool {
+	var got, want interface{}
+	switch c.Urn {
+	case "beam:coder:bytes:v1":
+		got = string(elem.Elm.([]byte))
+		switch egv := eg.Value.(type) {
+		case string:
+			want = egv
+		case []byte:
+			want = string(egv)
+		}
+	case "beam:coder:varint:v1":
+		got, want = elem.Elm.(int64), int64(eg.Value.(int))
+	case "beam:coder:double:v1":
+		got = elem.Elm.(float64)
+		switch v := eg.Value.(string); v {
+		case "NaN":
+			// Do the NaN comparison here since NaN by definition != NaN.
+			if math.IsNaN(got.(float64)) {
+				want, got = 1, 1
+			} else {
+				want = math.NaN()
+			}
+		case "-Infinity":
+			want = math.Inf(-1)
+		case "Infinity":
+			want = math.Inf(1)
+		default:
+			want, _ = strconv.ParseFloat(v, 64)
+		}
+
+	case "beam:coder:kv:v1":
+		v := eg.Value.(yaml.MapSlice)
+		pass := true
+		if !diff(c.Components[0], &exec.FullValue{Elm: elem.Elm}, v[0]) {
+			pass = false
+		}
+		if !diff(c.Components[1], &exec.FullValue{Elm: elem.Elm2}, v[1]) {
+			pass = false
+		}
+		return pass
+
+	case "beam:coder:iterable:v1":
+		pass := true
+		gotrv := reflect.ValueOf(elem.Elm)
+		wantrv := reflect.ValueOf(eg.Value)
+		if gotrv.Len() != wantrv.Len() {
+			log.Printf("Lengths don't match. got %v, want %v;  %v, %v", gotrv.Len(), wantrv.Len(), gotrv, wantrv)
+			return false
+		}
+		for i := 0; i < wantrv.Len(); i++ {
+			if !diff(c.Components[0],
+				&exec.FullValue{Elm: gotrv.Index(i).Interface()},
+				yaml.MapItem{Value: wantrv.Index(i).Interface()}) {
+				pass = false
+			}
+
+		}
+		return pass
+	case "beam:coder:interval_window:v1":
+		var a, b int
+		val := eg.Value
+		if is, ok := eg.Value.([]interface{}); ok {
+			val = is[0]
+		}
+		v := val.(yaml.MapSlice)
+
+		a = v[0].Value.(int)
+		b = v[1].Value.(int)
+		end := mtime.FromMilliseconds(int64(a))
+		start := end - mtime.Time(int64(b))
+		want = window.IntervalWindow{Start: start, End: end}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = elem.Elm
+		} else {
+			got = elem.Windows[0]
+		}
+
+	case "beam:coder:global_window:v1":
+		want = window.GlobalWindow{}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = window.GlobalWindow(elem.Elm.(struct{}))
+		} else {
+			got = elem.Windows[0]
+		}
+	case "beam:coder:windowed_value:v1", "beam:coder:param_windowed_value:v1":
+		// elem contains all the information, but we need to compare the element+timestamp
+		// separately from the windows, to avoid repeated expected value parsing logic.
+		pass := true
+		vs := eg.Value.(yaml.MapSlice)
+		if !diff(c.Components[0], elem, vs[0]) {
+			pass = false
+		}
+		if d := cmp.Diff(
+			mtime.FromMilliseconds(int64(vs[1].Value.(int))),
+			elem.Timestamp, cmpOpts...); d != "" {
+
+			pass = false
+		}
+		if !diff(c.Components[1], elem, vs[3]) {
+			pass = false
+		}
+		// TODO compare pane information.
+		return pass
+	case "beam:coder:row:v1":
+		fs := eg.Value.(yaml.MapSlice)
+		var rfs []reflect.StructField
+		// There are only 2 pointer examples, but they reuse field names,
+		// so we key off the proto hash to know which example we're handling.
+		ptrEg := strings.Contains(c.Payload, "51ace21c7393")
+		for _, rf := range fs {
+			name := rf.Key.(string)
+			t := nameToType[name]
+			if ptrEg {
+				t = reflect.PtrTo(t)
+			}
+			rfs = append(rfs, reflect.StructField{
+				Name: strings.ToUpper(name[:1]) + name[1:],
+				Type: t,
+				Tag:  reflect.StructTag(fmt.Sprintf("beam:\"%v\"", name)),
+			})
+		}
+		rv := reflect.New(reflect.StructOf(rfs)).Elem()
+		for i, rf := range fs {
+			setField(rv, i, rf.Value)
+		}
+
+		got, want = elem.Elm, rv.Interface()
+	default:
+		got, want = elem.Elm, eg.Value
+	}
+	if d := cmp.Diff(want, got, cmpOpts...); d != "" {
+		log.Printf("Decoding error: diff(-want,+got): %v\n", d)
+		return false
+	}
+	return true
+}
+
+// standard_coders.yaml uses the name for type indication, except for nullability.
+var nameToType = map[string]reflect.Type{
+	"str":     reflectx.String,
+	"i32":     reflectx.Int32,
+	"f64":     reflectx.Float64,
+	"arr":     reflect.SliceOf(reflectx.String),
+	"f_bool":  reflectx.Bool,
+	"f_bytes": reflect.PtrTo(reflectx.ByteSlice),
+	"f_map":   reflect.MapOf(reflectx.String, reflect.PtrTo(reflectx.Int64)),
+}
+
+func setField(rv reflect.Value, i int, v interface{}) {
+	if v == nil {
+		return
+	}
+	rf := rv.Field(i)
+	if rf.Kind() == reflect.Ptr {
+		// Ensure it's initialized.
+		rf.Set(reflect.New(rf.Type().Elem()))
+		rf = rf.Elem()
+	}
+	switch rf.Kind() {
+	case reflect.String:
+		rf.SetString(v.(string))
+	case reflect.Int32:
+		rf.SetInt(int64(v.(int)))
+	case reflect.Float64:
+		c, err := strconv.ParseFloat(v.(string), 64)
+		if err != nil {
+			panic(err)
+		}
+		rf.SetFloat(c)
+	case reflect.Slice:
+		if rf.Type() == reflectx.ByteSlice {
+			rf.Set(reflect.ValueOf([]byte(v.(string))))
+			break
+		}
+		// Value is a []interface{} with string values.

Review comment:
       Is this assumption made specifically based on the yaml file this is meant to be used with? Or is this based on something in the schema spec I'm forgetting?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)
+		if c.Custom.Name != "schema" {
+			return dec
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.
+		switch c.T.Type().Kind() {
+		case reflect.Slice:
+			return &lpDecoder{
+				dec: &iterableDecoder{
+					t:   c.Custom.Type,
+					dec: dec,
+				},
+			}
+		case reflect.Array:
+			return &lpDecoder{
+				dec: &arrayDecoder{

Review comment:
       Just to confirm, using an `arrayDecoder` without an `arrayEncoder` specifically works because the format that `iterableEncoder` encodes to is also decodable as an array, right? That might be worth mentioning as a comment, because I was wondering why you can decode arrays with an `arrayDecoder` if they were encoded with an `iterableEncoder`.

##########
File path: sdks/go/pkg/beam/core/runtime/graphx/coder.go
##########
@@ -249,31 +256,35 @@ func (b *CoderUnmarshaller) makeCoder(c *pipepb.Coder) (*coder.Coder, error) {
 			return nil, err
 		}
 
-		// No payload means this coder was length prefixed by the runner
-		// but is likely self describing - AKA a beam coder.
-		if len(sub.GetSpec().GetPayload()) == 0 {
-			return b.makeCoder(sub)
-		}
 		// TODO(lostluck) 2018/10/17: Make this strict again, once dataflow can use
 		// the portable pipeline model directly (BEAM-2885)
-		if sub.GetSpec().GetUrn() != "" && sub.GetSpec().GetUrn() != urnCustomCoder {
-			// TODO(herohde) 11/17/2017: revisit this restriction
-			return nil, errors.Errorf("could not unmarshal length prefix coder from %v, want a custom coder as a sub component but got %v", c, sub)
-		}
-
-		var ref v1pb.CustomCoder
-		if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
-			return nil, err
-		}
-		custom, err := decodeCustomCoder(&ref)
-		if err != nil {
-			return nil, err
+		switch u := sub.GetSpec().GetUrn(); u {
+		case "", urnCustomCoder:
+			var ref v1pb.CustomCoder
+			if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
+				return nil, err
+			}
+			custom, err := decodeCustomCoder(&ref)
+			if err != nil {
+				return nil, err
+			}
+			custom.ID = components[0]
+			t := typex.New(custom.Type)
+			cc := &coder.Coder{Kind: coder.Custom, T: t, Custom: custom}
+			fmt.Println("decoded customcoder", cc)

Review comment:
       Looks like a debug Println left in.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)

Review comment:
       Is it necessary to cast n to an int here? Isn't it already an int from line 638?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {
+		err := c.dec.DecodeTo(r, &e)
+		if err != nil {
+			return reflect.Value{}, err
+		}
+		if e.Elm != nil {
+			rv.Index(i).Set(reflect.ValueOf(e.Elm))
+		} else {
+			rv.Index(i).Set(reflect.ValueOf(e.Windows[0]))

Review comment:
       I don't get why it's setting the value of the element to `e.Windows[0]`. Is it a way to pass in nil?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {

Review comment:
       Likewise, is it necessary to cast n here too?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] youngoli commented on a change in pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #12588:
URL: https://github.com/apache/beam/pull/12588#discussion_r489137571



##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -81,24 +82,82 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder {
 		return &stringEncoder{}
 
 	case coder.Custom:
-		return &customEncoder{
+		enc := &customEncoder{
 			t:   c.Custom.Type,
 			enc: makeEncoder(c.Custom.Enc.Fn),
 		}
+		if c.Custom.Name != "schema" {
+			return enc
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.

Review comment:
       Yeah that makes more sense to me.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-693700908


   No worries. I assumed a conflict would happen somewhere. I'm just glad it
   was caught before merging.
   Supported/ignored labels/urns would be the way to go, but It does mean
   redundancy in the test cases making them a touch harder to add/maintain.
   This one was caught out since it's about a feature inside the schema coder
   spec that can't be detected by the base URN.
   
   Perhaps a new optional field? Most of them can be caught by the URNs
   themselves without special handling but it's the nested feature support
   that's trickier to work around (like Go looking up "logical" in the
   payload).
   
   On Wed, Sep 16, 2020, 3:26 PM Brian Hulette <no...@github.com>
   wrote:
   
   > Sorry about that @lostluck <https://github.com/lostluck>, I should've let
   > you know about the new test case. There should probably be an easier way
   > for SDKs to temporarily opt out of particular tests. WDYT about giving each
   > suite labels that SDKs can use in an ignorelist?
   >
   > —
   > You are receiving this because you were mentioned.
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/beam/pull/12588#issuecomment-693698195>, or
   > unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/ADKDOFOEGXV6FDCALZF6JNDSGE3Q5ANCNFSM4P73YZXQ>
   > .
   >
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/c8991ba9b65af2f547bad68a0ac327ce720eb00b?el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #12588      +/-   ##
   ==========================================
   - Coverage   82.30%   82.30%   -0.01%     
   ==========================================
     Files         451      451              
     Lines       53818    53818              
   ==========================================
   - Hits        44294    44293       -1     
   - Misses       9524     9525       +1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/coders/row\_coder.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vY29kZXJzL3Jvd19jb2Rlci5weQ==) | `93.66% <0.00%> (-0.71%)` | :arrow_down: |
   | [sdks/python/apache\_beam/io/iobase.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW8vaW9iYXNlLnB5) | `84.04% <0.00%> (-0.29%)` | :arrow_down: |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `89.52% <0.00%> (+0.17%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [c8991ba...4d5eaa2](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-693561961


   PTAL


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/c8991ba9b65af2f547bad68a0ac327ce720eb00b?el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #12588      +/-   ##
   ==========================================
   - Coverage   82.30%   82.29%   -0.01%     
   ==========================================
     Files         451      451              
     Lines       53818    53818              
   ==========================================
   - Hits        44294    44289       -5     
   - Misses       9524     9529       +5     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/coders/row\_coder.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vY29kZXJzL3Jvd19jb2Rlci5weQ==) | `93.66% <0.00%> (-0.71%)` | :arrow_down: |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `88.98% <0.00%> (-0.36%)` | :arrow_down: |
   | [...hon/apache\_beam/runners/worker/bundle\_processor.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvYnVuZGxlX3Byb2Nlc3Nvci5weQ==) | `94.18% <0.00%> (-0.27%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [c8991ba...4d5eaa2](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/4635afbf686b216983d1c0552b33cd435f922812?el=desc) will **decrease** coverage by `0.04%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #12588      +/-   ##
   ==========================================
   - Coverage   40.42%   40.38%   -0.05%     
   ==========================================
     Files         450      451       +1     
     Lines       53665    53721      +56     
   ==========================================
     Hits        21696    21696              
   - Misses      31969    32025      +56     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...apache\_beam/runners/dataflow/internal/apiclient.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9hcGljbGllbnQucHk=) | `20.20% <0.00%> (ø)` | |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `28.06% <0.00%> (+0.04%)` | :arrow_up: |
   | [...eam/testing/benchmarks/nexmark/nexmark\_launcher.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya19sYXVuY2hlci5weQ==) | `0.00% <0.00%> (ø)` | |
   | [...he\_beam/testing/benchmarks/nexmark/nexmark\_perf.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya19wZXJmLnB5) | `0.00% <0.00%> (ø)` | |
   | [...he\_beam/testing/benchmarks/nexmark/nexmark\_util.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya191dGlsLnB5) | `0.00% <0.00%> (ø)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [6213d47...01095f0](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-693657844


   All the jenkins tests are passing, and this doesn't change any python code, so the failing python checks are likely spurious.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-690891253


   Thanks! I appreciate that.
   
   On Thu, Sep 10, 2020, 10:46 PM Daniel Oliveira <no...@github.com>
   wrote:
   
   > Progress update: Started reviewing this today, about 2/3 of the way
   > through. Wanted to let you know I was working on it, so you don't just get
   > radio silence.
   >
   > —
   > You are receiving this because you authored the thread.
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/beam/pull/12588#issuecomment-690889052>, or
   > unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/ADKDOFKIXR7XAR5AZRUIOZTSFG2TTANCNFSM4P73YZXQ>
   > .
   >
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] youngoli commented on a change in pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #12588:
URL: https://github.com/apache/beam/pull/12588#discussion_r489139814



##########
File path: sdks/go/test/regression/coders/fromyaml/fromyaml.go
##########
@@ -0,0 +1,415 @@
+// 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.
+
+// fromyaml generates a resource file from the standard_coders.yaml
+// file for use in these coder regression tests.
+//
+// It expects to be run in it's test directory, or via it's go test.
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"math"
+	"reflect"
+	"runtime/debug"
+	"strconv"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/google/go-cmp/cmp"
+	"golang.org/x/text/encoding/charmap"
+	yaml "gopkg.in/yaml.v2"
+)
+
+var unimplementedCoders = map[string]bool{
+	"beam:coder:param_windowed_value:v1": true,
+	"beam:coder:timer:v1":                true,
+}
+
+// Coder is a representation a serialized beam coder.
+type Coder struct {
+	Urn              string  `yaml:"urn,omitempty"`
+	Payload          string  `yaml:"payload,omitempty"`
+	Components       []Coder `yaml:"components,omitempty"`
+	NonDeterministic bool    `yaml:"non_deterministic,omitempty"`
+}
+
+type logger interface {
+	Errorf(string, ...interface{})
+	Logf(string, ...interface{})
+}
+
+// Spec is a set of conditions that a coder must pass.
+type Spec struct {
+	Coder    Coder         `yaml:"coder,omitempty"`
+	Nested   *bool         `yaml:"nested,omitempty"`
+	Examples yaml.MapSlice `yaml:"examples,omitempty"`
+	Log      logger
+
+	id       int // for generating coder ids.
+	coderPBs map[string]*pipepb.Coder
+}
+
+func (s *Spec) nextID() string {
+	ret := fmt.Sprintf("%d", s.id)
+	s.id++
+	return ret
+}
+
+func (s *Spec) testStandardCoder() (err error) {
+	if unimplementedCoders[s.Coder.Urn] {
+		log.Printf("skipping unimplemented coder urn: %v", s.Coder.Urn)
+		return nil
+	}
+	// Construct the coder proto equivalents.
+
+	// Only nested tests need to be run, since nestedness is a pre-portability
+	// concept.
+	// For legacy Java reasons, the row coder examples are all marked nested: false
+	// so we need to check that before skipping unnested tests.
+	if s.Coder.Urn != "beam:coder:row:v1" && s.Nested != nil && !*s.Nested {
+		log.Printf("skipping unnested coder spec: %v\n", s.Coder)
+		return nil
+	}
+
+	s.coderPBs = make(map[string]*pipepb.Coder)
+	id := s.parseCoder(s.Coder)
+	b := graphx.NewCoderUnmarshaller(s.coderPBs)
+	underTest, err := b.Coder(id)
+	if err != nil {
+		return fmt.Errorf("unable to create coder: %v", err)
+	}
+
+	defer func() {
+		if e := recover(); e != nil {
+			err = fmt.Errorf("panicked on coder %v || %v:\n\t%v :\n%s", underTest, s.Coder, e, debug.Stack())
+		}
+	}()
+
+	var decFails, encFails int
+	for _, eg := range s.Examples {
+
+		// Test Decoding
+		// Ideally we'd use the beam package coders, but KVs make that complicated.
+		// This can be cleaned up once a type parametered beam.KV type exists.
+		dec := exec.MakeElementDecoder(underTest)
+		encoded := eg.Key.(string)
+		var elem exec.FullValue
+
+		// What I would have expected.
+		//		r := charmap.ISO8859_1.NewDecoder().Reader(strings.NewReader(encoded))
+		recoded, err := charmap.ISO8859_1.NewEncoder().String(encoded)
+		if err != nil {
+			return err
+		}
+		r := strings.NewReader(recoded)
+		if err := dec.DecodeTo(r, &elem); err != nil {
+			return fmt.Errorf("err decoding %q: %v", encoded, err)
+		}
+		if !diff(s.Coder, &elem, eg) {
+			decFails++
+			continue
+		}
+
+		// Test Encoding
+		if s.Coder.NonDeterministic {
+			// Skip verifying nondeterministic encodings.
+			continue
+		}
+		enc := exec.MakeElementEncoder(underTest)
+		var out bytes.Buffer
+		if err := enc.Encode(&elem, &out); err != nil {
+			return err
+		}
+		if d := cmp.Diff(recoded, string(out.Bytes())); d != "" {
+			log.Printf("Encoding error: diff(-want,+got): %v\n", d)
+		}
+	}
+	if decFails+encFails > 0 {
+		return fmt.Errorf("failed to decode %v times, and encode %v times", decFails, encFails)
+	}
+
+	return nil
+}
+
+var cmpOpts = []cmp.Option{
+	cmp.Transformer("bytes2string", func(in []byte) (out string) {
+		return string(in)
+	}),
+}
+
+func diff(c Coder, elem *exec.FullValue, eg yaml.MapItem) bool {
+	var got, want interface{}
+	switch c.Urn {
+	case "beam:coder:bytes:v1":
+		got = string(elem.Elm.([]byte))
+		switch egv := eg.Value.(type) {
+		case string:
+			want = egv
+		case []byte:
+			want = string(egv)
+		}
+	case "beam:coder:varint:v1":
+		got, want = elem.Elm.(int64), int64(eg.Value.(int))
+	case "beam:coder:double:v1":
+		got = elem.Elm.(float64)
+		switch v := eg.Value.(string); v {
+		case "NaN":
+			// Do the NaN comparison here since NaN by definition != NaN.
+			if math.IsNaN(got.(float64)) {
+				want, got = 1, 1
+			} else {
+				want = math.NaN()
+			}
+		case "-Infinity":
+			want = math.Inf(-1)
+		case "Infinity":
+			want = math.Inf(1)
+		default:
+			want, _ = strconv.ParseFloat(v, 64)
+		}
+
+	case "beam:coder:kv:v1":
+		v := eg.Value.(yaml.MapSlice)
+		pass := true
+		if !diff(c.Components[0], &exec.FullValue{Elm: elem.Elm}, v[0]) {
+			pass = false
+		}
+		if !diff(c.Components[1], &exec.FullValue{Elm: elem.Elm2}, v[1]) {
+			pass = false
+		}
+		return pass
+
+	case "beam:coder:iterable:v1":
+		pass := true
+		gotrv := reflect.ValueOf(elem.Elm)
+		wantrv := reflect.ValueOf(eg.Value)
+		if gotrv.Len() != wantrv.Len() {
+			log.Printf("Lengths don't match. got %v, want %v;  %v, %v", gotrv.Len(), wantrv.Len(), gotrv, wantrv)
+			return false
+		}
+		for i := 0; i < wantrv.Len(); i++ {
+			if !diff(c.Components[0],
+				&exec.FullValue{Elm: gotrv.Index(i).Interface()},
+				yaml.MapItem{Value: wantrv.Index(i).Interface()}) {
+				pass = false
+			}
+
+		}
+		return pass
+	case "beam:coder:interval_window:v1":
+		var a, b int
+		val := eg.Value
+		if is, ok := eg.Value.([]interface{}); ok {
+			val = is[0]
+		}
+		v := val.(yaml.MapSlice)
+
+		a = v[0].Value.(int)
+		b = v[1].Value.(int)
+		end := mtime.FromMilliseconds(int64(a))
+		start := end - mtime.Time(int64(b))
+		want = window.IntervalWindow{Start: start, End: end}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = elem.Elm
+		} else {
+			got = elem.Windows[0]
+		}
+
+	case "beam:coder:global_window:v1":
+		want = window.GlobalWindow{}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = window.GlobalWindow(elem.Elm.(struct{}))
+		} else {
+			got = elem.Windows[0]
+		}
+	case "beam:coder:windowed_value:v1", "beam:coder:param_windowed_value:v1":
+		// elem contains all the information, but we need to compare the element+timestamp
+		// separately from the windows, to avoid repeated expected value parsing logic.
+		pass := true
+		vs := eg.Value.(yaml.MapSlice)
+		if !diff(c.Components[0], elem, vs[0]) {
+			pass = false
+		}
+		if d := cmp.Diff(
+			mtime.FromMilliseconds(int64(vs[1].Value.(int))),
+			elem.Timestamp, cmpOpts...); d != "" {
+
+			pass = false
+		}
+		if !diff(c.Components[1], elem, vs[3]) {
+			pass = false
+		}
+		// TODO compare pane information.
+		return pass
+	case "beam:coder:row:v1":
+		fs := eg.Value.(yaml.MapSlice)
+		var rfs []reflect.StructField
+		// There are only 2 pointer examples, but they reuse field names,
+		// so we key off the proto hash to know which example we're handling.
+		ptrEg := strings.Contains(c.Payload, "51ace21c7393")
+		for _, rf := range fs {
+			name := rf.Key.(string)
+			t := nameToType[name]
+			if ptrEg {
+				t = reflect.PtrTo(t)
+			}
+			rfs = append(rfs, reflect.StructField{
+				Name: strings.ToUpper(name[:1]) + name[1:],
+				Type: t,
+				Tag:  reflect.StructTag(fmt.Sprintf("beam:\"%v\"", name)),
+			})
+		}
+		rv := reflect.New(reflect.StructOf(rfs)).Elem()
+		for i, rf := range fs {
+			setField(rv, i, rf.Value)
+		}
+
+		got, want = elem.Elm, rv.Interface()
+	default:
+		got, want = elem.Elm, eg.Value
+	}
+	if d := cmp.Diff(want, got, cmpOpts...); d != "" {
+		log.Printf("Decoding error: diff(-want,+got): %v\n", d)
+		return false
+	}
+	return true
+}
+
+// standard_coders.yaml uses the name for type indication, except for nullability.
+var nameToType = map[string]reflect.Type{
+	"str":     reflectx.String,
+	"i32":     reflectx.Int32,
+	"f64":     reflectx.Float64,
+	"arr":     reflect.SliceOf(reflectx.String),
+	"f_bool":  reflectx.Bool,
+	"f_bytes": reflect.PtrTo(reflectx.ByteSlice),
+	"f_map":   reflect.MapOf(reflectx.String, reflect.PtrTo(reflectx.Int64)),
+}
+
+func setField(rv reflect.Value, i int, v interface{}) {
+	if v == nil {
+		return
+	}
+	rf := rv.Field(i)
+	if rf.Kind() == reflect.Ptr {
+		// Ensure it's initialized.
+		rf.Set(reflect.New(rf.Type().Elem()))
+		rf = rf.Elem()
+	}
+	switch rf.Kind() {
+	case reflect.String:
+		rf.SetString(v.(string))
+	case reflect.Int32:
+		rf.SetInt(int64(v.(int)))
+	case reflect.Float64:
+		c, err := strconv.ParseFloat(v.(string), 64)
+		if err != nil {
+			panic(err)
+		}
+		rf.SetFloat(c)
+	case reflect.Slice:
+		if rf.Type() == reflectx.ByteSlice {
+			rf.Set(reflect.ValueOf([]byte(v.(string))))
+			break
+		}
+		// Value is a []interface{} with string values.

Review comment:
       Gotcha. It seems a little brittle (if the yaml file ever adds an example with a list of other types then this will break), so a comment mentioning that it's due to the contents of the yaml file would be helpful if that happens.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-690891253


   Thanks! I appreciate that.
   
   On Thu, Sep 10, 2020, 10:46 PM Daniel Oliveira <no...@github.com>
   wrote:
   
   > Progress update: Started reviewing this today, about 2/3 of the way
   > through. Wanted to let you know I was working on it, so you don't just get
   > radio silence.
   >
   > —
   > You are receiving this because you authored the thread.
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/beam/pull/12588#issuecomment-690889052>, or
   > unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/ADKDOFKIXR7XAR5AZRUIOZTSFG2TTANCNFSM4P73YZXQ>
   > .
   >
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] youngoli commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
youngoli commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-690889052


   Progress update: Started reviewing this today, about 2/3 of the way through. Wanted to let you know I was working on it, so you don't just get radio silence.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > :exclamation: No coverage uploaded for pull request head (`beam9615a@01095f0`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference#section-missing-head-commit).
   > The diff coverage is `n/a`.
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/2bb60c323095340240ec4982c1e9dabc397107e5?el=desc) will **decrease** coverage by `0.04%`.
   > The diff coverage is `20.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #12588      +/-   ##
   ==========================================
   - Coverage   40.43%   40.38%   -0.05%     
   ==========================================
     Files         449      451       +2     
     Lines       53530    53721     +191     
   ==========================================
   + Hits        21643    21696      +53     
   - Misses      31887    32025     +138     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/io/gcp/bigquery\_tools.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW8vZ2NwL2JpZ3F1ZXJ5X3Rvb2xzLnB5) | `28.57% <0.00%> (-0.19%)` | :arrow_down: |
   | [...apache\_beam/runners/dataflow/internal/apiclient.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9hcGljbGllbnQucHk=) | `20.20% <0.00%> (ø)` | |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `28.06% <0.00%> (+0.04%)` | :arrow_up: |
   | [...eam/testing/benchmarks/nexmark/nexmark\_launcher.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya19sYXVuY2hlci5weQ==) | `0.00% <0.00%> (ø)` | |
   | [...he\_beam/testing/benchmarks/nexmark/nexmark\_perf.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya19wZXJmLnB5) | `0.00% <0.00%> (ø)` | |
   | [...he\_beam/testing/benchmarks/nexmark/nexmark\_util.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya191dGlsLnB5) | `0.00% <0.00%> (ø)` | |
   | [sdks/python/apache\_beam/io/gcp/bigquery.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW8vZ2NwL2JpZ3F1ZXJ5LnB5) | `27.10% <38.88%> (+0.33%)` | :arrow_up: |
   | [sdks/python/apache\_beam/utils/histogram.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdXRpbHMvaGlzdG9ncmFtLnB5) | `40.00% <40.00%> (ø)` | |
   | [...dks/python/apache\_beam/options/pipeline\_options.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vb3B0aW9ucy9waXBlbGluZV9vcHRpb25zLnB5) | `55.98% <50.00%> (-0.14%)` | :arrow_down: |
   | ... and [1 more](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [6213d47...c8f2cdf](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-690891253






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on a change in pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on a change in pull request #12588:
URL: https://github.com/apache/beam/pull/12588#discussion_r489607556



##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)

Review comment:
       Done.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {

Review comment:
       Done.

##########
File path: sdks/go/pkg/beam/core/runtime/graphx/coder.go
##########
@@ -249,31 +256,35 @@ func (b *CoderUnmarshaller) makeCoder(c *pipepb.Coder) (*coder.Coder, error) {
 			return nil, err
 		}
 
-		// No payload means this coder was length prefixed by the runner
-		// but is likely self describing - AKA a beam coder.
-		if len(sub.GetSpec().GetPayload()) == 0 {
-			return b.makeCoder(sub)
-		}
 		// TODO(lostluck) 2018/10/17: Make this strict again, once dataflow can use
 		// the portable pipeline model directly (BEAM-2885)
-		if sub.GetSpec().GetUrn() != "" && sub.GetSpec().GetUrn() != urnCustomCoder {
-			// TODO(herohde) 11/17/2017: revisit this restriction
-			return nil, errors.Errorf("could not unmarshal length prefix coder from %v, want a custom coder as a sub component but got %v", c, sub)
-		}
-
-		var ref v1pb.CustomCoder
-		if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
-			return nil, err
-		}
-		custom, err := decodeCustomCoder(&ref)
-		if err != nil {
-			return nil, err
+		switch u := sub.GetSpec().GetUrn(); u {
+		case "", urnCustomCoder:
+			var ref v1pb.CustomCoder
+			if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
+				return nil, err
+			}
+			custom, err := decodeCustomCoder(&ref)
+			if err != nil {
+				return nil, err
+			}
+			custom.ID = components[0]
+			t := typex.New(custom.Type)
+			cc := &coder.Coder{Kind: coder.Custom, T: t, Custom: custom}
+			fmt.Println("decoded customcoder", cc)

Review comment:
       Done.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)
+		if c.Custom.Name != "schema" {
+			return dec
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.
+		switch c.T.Type().Kind() {
+		case reflect.Slice:
+			return &lpDecoder{
+				dec: &iterableDecoder{
+					t:   c.Custom.Type,
+					dec: dec,
+				},
+			}
+		case reflect.Array:
+			return &lpDecoder{
+				dec: &arrayDecoder{

Review comment:
       That's right the encoding format for both is the same. I'll put the comment on the array decoder itself rather than here though.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -534,26 +1038,34 @@ func (*intervalWindowEncoder) EncodeSingle(elm typex.Window, w io.Writer) error
 
 type intervalWindowDecoder struct{}
 
-func (*intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
+func (d *intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
 	// Encoding: upper bound and duration
 
 	n, err := coder.DecodeInt32(r) // #windows
 
 	ret := make([]typex.Window, n, n)
 	for i := int32(0); i < n; i++ {
-		end, err := coder.DecodeEventTime(r)
-		if err != nil {
-			return nil, err
-		}
-		duration, err := coder.DecodeVarUint64(r)
+		w, err := d.DecodeSingle(r)
 		if err != nil {
 			return nil, err
 		}
-		ret[i] = window.IntervalWindow{Start: mtime.FromMilliseconds(end.Milliseconds() - int64(duration)), End: end}
+		ret[i] = w
 	}
 	return ret, err
 }
 
+func (*intervalWindowDecoder) DecodeSingle(r io.Reader) (typex.Window, error) {
+	end, err := coder.DecodeEventTime(r)
+	if err != nil {
+		return nil, err
+	}
+	duration, err := coder.DecodeVarInt(r)

Review comment:
       Ah no difference TBH. DecodeVarInt basically just does an extra cast from uint64 to int64, which I didn't remove. 
   I went back to `DecodeVarUint64` since it avoids an extra layer of if checks and a function call.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-675655446


   R: @youngoli  
   
   I see the new test failed, I don't anticipate any major changes while I investigate and fix that. PTAL.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/c8991ba9b65af2f547bad68a0ac327ce720eb00b?el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #12588      +/-   ##
   ==========================================
   - Coverage   82.30%   82.29%   -0.01%     
   ==========================================
     Files         451      451              
     Lines       53818    53818              
   ==========================================
   - Hits        44294    44289       -5     
   - Misses       9524     9529       +5     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/coders/row\_coder.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vY29kZXJzL3Jvd19jb2Rlci5weQ==) | `93.66% <0.00%> (-0.71%)` | :arrow_down: |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `88.98% <0.00%> (-0.36%)` | :arrow_down: |
   | [...hon/apache\_beam/runners/worker/bundle\_processor.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvYnVuZGxlX3Byb2Nlc3Nvci5weQ==) | `94.18% <0.00%> (-0.27%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [c8991ba...4d5eaa2](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/c8991ba9b65af2f547bad68a0ac327ce720eb00b?el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #12588      +/-   ##
   ==========================================
   - Coverage   82.30%   82.30%   -0.01%     
   ==========================================
     Files         451      451              
     Lines       53818    53818              
   ==========================================
   - Hits        44294    44293       -1     
   - Misses       9524     9525       +1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/coders/row\_coder.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vY29kZXJzL3Jvd19jb2Rlci5weQ==) | `93.66% <0.00%> (-0.71%)` | :arrow_down: |
   | [sdks/python/apache\_beam/io/iobase.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW8vaW9iYXNlLnB5) | `84.04% <0.00%> (-0.29%)` | :arrow_down: |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `89.52% <0.00%> (+0.17%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [c8991ba...4f42690](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] youngoli commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
youngoli commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-690889052


   Progress update: Started reviewing this today, about 2/3 of the way through. Wanted to let you know I was working on it, so you don't just get radio silence.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/c8991ba9b65af2f547bad68a0ac327ce720eb00b?el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #12588      +/-   ##
   ==========================================
   - Coverage   82.30%   82.30%   -0.01%     
   ==========================================
     Files         451      451              
     Lines       53818    53818              
   ==========================================
   - Hits        44294    44293       -1     
   - Misses       9524     9525       +1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/coders/row\_coder.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vY29kZXJzL3Jvd19jb2Rlci5weQ==) | `93.66% <0.00%> (-0.71%)` | :arrow_down: |
   | [sdks/python/apache\_beam/io/iobase.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW8vaW9iYXNlLnB5) | `84.04% <0.00%> (-0.29%)` | :arrow_down: |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `89.52% <0.00%> (+0.17%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [c8991ba...4f42690](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-690891253


   Thanks! I appreciate that.
   
   On Thu, Sep 10, 2020, 10:46 PM Daniel Oliveira <no...@github.com>
   wrote:
   
   > Progress update: Started reviewing this today, about 2/3 of the way
   > through. Wanted to let you know I was working on it, so you don't just get
   > radio silence.
   >
   > —
   > You are receiving this because you authored the thread.
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/beam/pull/12588#issuecomment-690889052>, or
   > unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/ADKDOFKIXR7XAR5AZRUIOZTSFG2TTANCNFSM4P73YZXQ>
   > .
   >
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/4635afbf686b216983d1c0552b33cd435f922812?el=desc) will **decrease** coverage by `0.04%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #12588      +/-   ##
   ==========================================
   - Coverage   40.42%   40.38%   -0.05%     
   ==========================================
     Files         450      451       +1     
     Lines       53665    53721      +56     
   ==========================================
     Hits        21696    21696              
   - Misses      31969    32025      +56     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...apache\_beam/runners/dataflow/internal/apiclient.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9hcGljbGllbnQucHk=) | `20.20% <0.00%> (ø)` | |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `28.06% <0.00%> (+0.04%)` | :arrow_up: |
   | [...eam/testing/benchmarks/nexmark/nexmark\_launcher.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya19sYXVuY2hlci5weQ==) | `0.00% <0.00%> (ø)` | |
   | [...he\_beam/testing/benchmarks/nexmark/nexmark\_perf.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya19wZXJmLnB5) | `0.00% <0.00%> (ø)` | |
   | [...he\_beam/testing/benchmarks/nexmark/nexmark\_util.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya191dGlsLnB5) | `0.00% <0.00%> (ø)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [6213d47...01095f0](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/4635afbf686b216983d1c0552b33cd435f922812?el=desc) will **decrease** coverage by `0.04%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #12588      +/-   ##
   ==========================================
   - Coverage   40.42%   40.38%   -0.05%     
   ==========================================
     Files         450      451       +1     
     Lines       53665    53721      +56     
   ==========================================
     Hits        21696    21696              
   - Misses      31969    32025      +56     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...apache\_beam/runners/dataflow/internal/apiclient.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9hcGljbGllbnQucHk=) | `20.20% <0.00%> (ø)` | |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `28.06% <0.00%> (+0.04%)` | :arrow_up: |
   | [...eam/testing/benchmarks/nexmark/nexmark\_launcher.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya19sYXVuY2hlci5weQ==) | `0.00% <0.00%> (ø)` | |
   | [...he\_beam/testing/benchmarks/nexmark/nexmark\_perf.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya19wZXJmLnB5) | `0.00% <0.00%> (ø)` | |
   | [...he\_beam/testing/benchmarks/nexmark/nexmark\_util.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya191dGlsLnB5) | `0.00% <0.00%> (ø)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [6213d47...01095f0](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689714287


   R: @tysonjh @youngoli 
   cc: @TheNeuralBit 
   
   I'm finally getting around to fixing the build dep issue with this one, but that will not affect it's ability to be reviewed.
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-676525512


   Run Java PreCommit


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on a change in pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on a change in pull request #12588:
URL: https://github.com/apache/beam/pull/12588#discussion_r488222388



##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -81,24 +82,82 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder {
 		return &stringEncoder{}
 
 	case coder.Custom:
-		return &customEncoder{
+		enc := &customEncoder{
 			t:   c.Custom.Type,
 			enc: makeEncoder(c.Custom.Enc.Fn),
 		}
+		if c.Custom.Name != "schema" {
+			return enc
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.

Review comment:
       "Custom schema coding uses the beam row coder, but wrapping it in a length prefix and treats it as an opaque coder."
   
   Does that make more sense?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/4635afbf686b216983d1c0552b33cd435f922812?el=desc) will **decrease** coverage by `0.04%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #12588      +/-   ##
   ==========================================
   - Coverage   40.42%   40.38%   -0.05%     
   ==========================================
     Files         450      451       +1     
     Lines       53665    53721      +56     
   ==========================================
     Hits        21696    21696              
   - Misses      31969    32025      +56     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...apache\_beam/runners/dataflow/internal/apiclient.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9hcGljbGllbnQucHk=) | `20.20% <0.00%> (ø)` | |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `28.06% <0.00%> (+0.04%)` | :arrow_up: |
   | [...eam/testing/benchmarks/nexmark/nexmark\_launcher.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya19sYXVuY2hlci5weQ==) | `0.00% <0.00%> (ø)` | |
   | [...he\_beam/testing/benchmarks/nexmark/nexmark\_perf.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya19wZXJmLnB5) | `0.00% <0.00%> (ø)` | |
   | [...he\_beam/testing/benchmarks/nexmark/nexmark\_util.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy9iZW5jaG1hcmtzL25leG1hcmsvbmV4bWFya191dGlsLnB5) | `0.00% <0.00%> (ø)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [6213d47...01095f0](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/c8991ba9b65af2f547bad68a0ac327ce720eb00b?el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #12588      +/-   ##
   ==========================================
   - Coverage   82.30%   82.30%   -0.01%     
   ==========================================
     Files         451      451              
     Lines       53818    53818              
   ==========================================
   - Hits        44294    44293       -1     
   - Misses       9524     9525       +1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/coders/row\_coder.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vY29kZXJzL3Jvd19jb2Rlci5weQ==) | `93.66% <0.00%> (-0.71%)` | :arrow_down: |
   | [sdks/python/apache\_beam/io/iobase.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW8vaW9iYXNlLnB5) | `84.04% <0.00%> (-0.29%)` | :arrow_down: |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `89.52% <0.00%> (+0.17%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [c8991ba...4f42690](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/c8991ba9b65af2f547bad68a0ac327ce720eb00b?el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #12588      +/-   ##
   ==========================================
   - Coverage   82.30%   82.30%   -0.01%     
   ==========================================
     Files         451      451              
     Lines       53818    53818              
   ==========================================
   - Hits        44294    44293       -1     
   - Misses       9524     9525       +1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/coders/row\_coder.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vY29kZXJzL3Jvd19jb2Rlci5weQ==) | `93.66% <0.00%> (-0.71%)` | :arrow_down: |
   | [sdks/python/apache\_beam/io/iobase.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW8vaW9iYXNlLnB5) | `84.04% <0.00%> (-0.29%)` | :arrow_down: |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `89.52% <0.00%> (+0.17%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [c8991ba...4f42690](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-676524987


   Run Go PreCommit


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck merged pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck merged pull request #12588:
URL: https://github.com/apache/beam/pull/12588


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-693622943


   It shouldn't be hard to fix, but apparently a new test case was added to standard_coders.yaml was added in the month this PR has sat around, causing it to break. I'll resolve and if it requires a significant change to this PR I'll let you know. Thanks!


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689814660


   Run Go PostCommit


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689721255


   Run Go PreCommit


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on a change in pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on a change in pull request #12588:
URL: https://github.com/apache/beam/pull/12588#discussion_r488227779



##########
File path: sdks/go/test/regression/coders/fromyaml/fromyaml.go
##########
@@ -0,0 +1,415 @@
+// 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.
+
+// fromyaml generates a resource file from the standard_coders.yaml
+// file for use in these coder regression tests.
+//
+// It expects to be run in it's test directory, or via it's go test.
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"math"
+	"reflect"
+	"runtime/debug"
+	"strconv"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/google/go-cmp/cmp"
+	"golang.org/x/text/encoding/charmap"
+	yaml "gopkg.in/yaml.v2"
+)
+
+var unimplementedCoders = map[string]bool{
+	"beam:coder:param_windowed_value:v1": true,
+	"beam:coder:timer:v1":                true,
+}
+
+// Coder is a representation a serialized beam coder.
+type Coder struct {
+	Urn              string  `yaml:"urn,omitempty"`
+	Payload          string  `yaml:"payload,omitempty"`
+	Components       []Coder `yaml:"components,omitempty"`
+	NonDeterministic bool    `yaml:"non_deterministic,omitempty"`
+}
+
+type logger interface {
+	Errorf(string, ...interface{})
+	Logf(string, ...interface{})
+}
+
+// Spec is a set of conditions that a coder must pass.
+type Spec struct {
+	Coder    Coder         `yaml:"coder,omitempty"`
+	Nested   *bool         `yaml:"nested,omitempty"`
+	Examples yaml.MapSlice `yaml:"examples,omitempty"`
+	Log      logger
+
+	id       int // for generating coder ids.
+	coderPBs map[string]*pipepb.Coder
+}
+
+func (s *Spec) nextID() string {
+	ret := fmt.Sprintf("%d", s.id)
+	s.id++
+	return ret
+}
+
+func (s *Spec) testStandardCoder() (err error) {
+	if unimplementedCoders[s.Coder.Urn] {
+		log.Printf("skipping unimplemented coder urn: %v", s.Coder.Urn)
+		return nil
+	}
+	// Construct the coder proto equivalents.
+
+	// Only nested tests need to be run, since nestedness is a pre-portability
+	// concept.
+	// For legacy Java reasons, the row coder examples are all marked nested: false
+	// so we need to check that before skipping unnested tests.
+	if s.Coder.Urn != "beam:coder:row:v1" && s.Nested != nil && !*s.Nested {
+		log.Printf("skipping unnested coder spec: %v\n", s.Coder)
+		return nil
+	}
+
+	s.coderPBs = make(map[string]*pipepb.Coder)
+	id := s.parseCoder(s.Coder)
+	b := graphx.NewCoderUnmarshaller(s.coderPBs)
+	underTest, err := b.Coder(id)
+	if err != nil {
+		return fmt.Errorf("unable to create coder: %v", err)
+	}
+
+	defer func() {
+		if e := recover(); e != nil {
+			err = fmt.Errorf("panicked on coder %v || %v:\n\t%v :\n%s", underTest, s.Coder, e, debug.Stack())
+		}
+	}()
+
+	var decFails, encFails int
+	for _, eg := range s.Examples {
+
+		// Test Decoding
+		// Ideally we'd use the beam package coders, but KVs make that complicated.
+		// This can be cleaned up once a type parametered beam.KV type exists.
+		dec := exec.MakeElementDecoder(underTest)
+		encoded := eg.Key.(string)
+		var elem exec.FullValue
+
+		// What I would have expected.
+		//		r := charmap.ISO8859_1.NewDecoder().Reader(strings.NewReader(encoded))
+		recoded, err := charmap.ISO8859_1.NewEncoder().String(encoded)
+		if err != nil {
+			return err
+		}
+		r := strings.NewReader(recoded)
+		if err := dec.DecodeTo(r, &elem); err != nil {
+			return fmt.Errorf("err decoding %q: %v", encoded, err)
+		}
+		if !diff(s.Coder, &elem, eg) {
+			decFails++
+			continue
+		}
+
+		// Test Encoding
+		if s.Coder.NonDeterministic {
+			// Skip verifying nondeterministic encodings.
+			continue
+		}
+		enc := exec.MakeElementEncoder(underTest)
+		var out bytes.Buffer
+		if err := enc.Encode(&elem, &out); err != nil {
+			return err
+		}
+		if d := cmp.Diff(recoded, string(out.Bytes())); d != "" {
+			log.Printf("Encoding error: diff(-want,+got): %v\n", d)
+		}
+	}
+	if decFails+encFails > 0 {
+		return fmt.Errorf("failed to decode %v times, and encode %v times", decFails, encFails)
+	}
+
+	return nil
+}
+
+var cmpOpts = []cmp.Option{
+	cmp.Transformer("bytes2string", func(in []byte) (out string) {
+		return string(in)
+	}),
+}
+
+func diff(c Coder, elem *exec.FullValue, eg yaml.MapItem) bool {
+	var got, want interface{}
+	switch c.Urn {
+	case "beam:coder:bytes:v1":
+		got = string(elem.Elm.([]byte))
+		switch egv := eg.Value.(type) {
+		case string:
+			want = egv
+		case []byte:
+			want = string(egv)
+		}
+	case "beam:coder:varint:v1":
+		got, want = elem.Elm.(int64), int64(eg.Value.(int))
+	case "beam:coder:double:v1":
+		got = elem.Elm.(float64)
+		switch v := eg.Value.(string); v {
+		case "NaN":
+			// Do the NaN comparison here since NaN by definition != NaN.
+			if math.IsNaN(got.(float64)) {
+				want, got = 1, 1
+			} else {
+				want = math.NaN()
+			}
+		case "-Infinity":
+			want = math.Inf(-1)
+		case "Infinity":
+			want = math.Inf(1)
+		default:
+			want, _ = strconv.ParseFloat(v, 64)
+		}
+
+	case "beam:coder:kv:v1":
+		v := eg.Value.(yaml.MapSlice)
+		pass := true
+		if !diff(c.Components[0], &exec.FullValue{Elm: elem.Elm}, v[0]) {
+			pass = false
+		}
+		if !diff(c.Components[1], &exec.FullValue{Elm: elem.Elm2}, v[1]) {
+			pass = false
+		}
+		return pass
+
+	case "beam:coder:iterable:v1":
+		pass := true
+		gotrv := reflect.ValueOf(elem.Elm)
+		wantrv := reflect.ValueOf(eg.Value)
+		if gotrv.Len() != wantrv.Len() {
+			log.Printf("Lengths don't match. got %v, want %v;  %v, %v", gotrv.Len(), wantrv.Len(), gotrv, wantrv)
+			return false
+		}
+		for i := 0; i < wantrv.Len(); i++ {
+			if !diff(c.Components[0],
+				&exec.FullValue{Elm: gotrv.Index(i).Interface()},
+				yaml.MapItem{Value: wantrv.Index(i).Interface()}) {
+				pass = false
+			}
+
+		}
+		return pass
+	case "beam:coder:interval_window:v1":
+		var a, b int
+		val := eg.Value
+		if is, ok := eg.Value.([]interface{}); ok {
+			val = is[0]
+		}
+		v := val.(yaml.MapSlice)
+
+		a = v[0].Value.(int)
+		b = v[1].Value.(int)
+		end := mtime.FromMilliseconds(int64(a))
+		start := end - mtime.Time(int64(b))
+		want = window.IntervalWindow{Start: start, End: end}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = elem.Elm
+		} else {
+			got = elem.Windows[0]
+		}
+
+	case "beam:coder:global_window:v1":
+		want = window.GlobalWindow{}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = window.GlobalWindow(elem.Elm.(struct{}))
+		} else {
+			got = elem.Windows[0]
+		}
+	case "beam:coder:windowed_value:v1", "beam:coder:param_windowed_value:v1":
+		// elem contains all the information, but we need to compare the element+timestamp
+		// separately from the windows, to avoid repeated expected value parsing logic.
+		pass := true
+		vs := eg.Value.(yaml.MapSlice)
+		if !diff(c.Components[0], elem, vs[0]) {
+			pass = false
+		}
+		if d := cmp.Diff(
+			mtime.FromMilliseconds(int64(vs[1].Value.(int))),
+			elem.Timestamp, cmpOpts...); d != "" {
+
+			pass = false
+		}
+		if !diff(c.Components[1], elem, vs[3]) {
+			pass = false
+		}
+		// TODO compare pane information.
+		return pass
+	case "beam:coder:row:v1":
+		fs := eg.Value.(yaml.MapSlice)
+		var rfs []reflect.StructField
+		// There are only 2 pointer examples, but they reuse field names,
+		// so we key off the proto hash to know which example we're handling.
+		ptrEg := strings.Contains(c.Payload, "51ace21c7393")
+		for _, rf := range fs {
+			name := rf.Key.(string)
+			t := nameToType[name]
+			if ptrEg {
+				t = reflect.PtrTo(t)
+			}
+			rfs = append(rfs, reflect.StructField{
+				Name: strings.ToUpper(name[:1]) + name[1:],
+				Type: t,
+				Tag:  reflect.StructTag(fmt.Sprintf("beam:\"%v\"", name)),
+			})
+		}
+		rv := reflect.New(reflect.StructOf(rfs)).Elem()
+		for i, rf := range fs {
+			setField(rv, i, rf.Value)
+		}
+
+		got, want = elem.Elm, rv.Interface()
+	default:
+		got, want = elem.Elm, eg.Value
+	}
+	if d := cmp.Diff(want, got, cmpOpts...); d != "" {
+		log.Printf("Decoding error: diff(-want,+got): %v\n", d)
+		return false
+	}
+	return true
+}
+
+// standard_coders.yaml uses the name for type indication, except for nullability.
+var nameToType = map[string]reflect.Type{
+	"str":     reflectx.String,
+	"i32":     reflectx.Int32,
+	"f64":     reflectx.Float64,
+	"arr":     reflect.SliceOf(reflectx.String),
+	"f_bool":  reflectx.Bool,
+	"f_bytes": reflect.PtrTo(reflectx.ByteSlice),
+	"f_map":   reflect.MapOf(reflectx.String, reflect.PtrTo(reflectx.Int64)),
+}
+
+func setField(rv reflect.Value, i int, v interface{}) {
+	if v == nil {
+		return
+	}
+	rf := rv.Field(i)
+	if rf.Kind() == reflect.Ptr {
+		// Ensure it's initialized.
+		rf.Set(reflect.New(rf.Type().Elem()))
+		rf = rf.Elem()
+	}
+	switch rf.Kind() {
+	case reflect.String:
+		rf.SetString(v.(string))
+	case reflect.Int32:
+		rf.SetInt(int64(v.(int)))
+	case reflect.Float64:
+		c, err := strconv.ParseFloat(v.(string), 64)
+		if err != nil {
+			panic(err)
+		}
+		rf.SetFloat(c)
+	case reflect.Slice:
+		if rf.Type() == reflectx.ByteSlice {
+			rf.Set(reflect.ValueOf([]byte(v.(string))))
+			break
+		}
+		// Value is a []interface{} with string values.

Review comment:
       The yaml file exclusively uses lists of strings, which is why we know what type to interpret them as here. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-674352299


   This branch was synced to the bad branch from earlier, hence the check failures. I'll rebase with not other changes on monday.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] codecov[bot] edited a comment on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-689776953


   # [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=h1) Report
   > Merging [#12588](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=desc) into [master](https://codecov.io/gh/apache/beam/commit/c8991ba9b65af2f547bad68a0ac327ce720eb00b?el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/12588/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1)](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #12588      +/-   ##
   ==========================================
   - Coverage   82.30%   82.29%   -0.01%     
   ==========================================
     Files         451      451              
     Lines       53818    53818              
   ==========================================
   - Hits        44294    44289       -5     
   - Misses       9524     9529       +5     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/coders/row\_coder.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vY29kZXJzL3Jvd19jb2Rlci5weQ==) | `93.66% <0.00%> (-0.71%)` | :arrow_down: |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `88.98% <0.00%> (-0.36%)` | :arrow_down: |
   | [...hon/apache\_beam/runners/worker/bundle\_processor.py](https://codecov.io/gh/apache/beam/pull/12588/diff?src=pr&el=tree#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvYnVuZGxlX3Byb2Nlc3Nvci5weQ==) | `94.18% <0.00%> (-0.27%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=footer). Last update [c8991ba...4d5eaa2](https://codecov.io/gh/apache/beam/pull/12588?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] youngoli commented on pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
youngoli commented on pull request #12588:
URL: https://github.com/apache/beam/pull/12588#issuecomment-690889052






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] youngoli commented on a change in pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #12588:
URL: https://github.com/apache/beam/pull/12588#discussion_r489138265



##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {
+		err := c.dec.DecodeTo(r, &e)
+		if err != nil {
+			return reflect.Value{}, err
+		}
+		if e.Elm != nil {
+			rv.Index(i).Set(reflect.ValueOf(e.Elm))
+		} else {
+			rv.Index(i).Set(reflect.ValueOf(e.Windows[0]))

Review comment:
       Ah that makes sense now. Yeah definitely could use a comment.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [beam] youngoli commented on a change in pull request #12588: [BEAM-7009] Add Go SDK Standard Coders yaml tests.

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #12588:
URL: https://github.com/apache/beam/pull/12588#discussion_r486767704



##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -81,24 +82,82 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder {
 		return &stringEncoder{}
 
 	case coder.Custom:
-		return &customEncoder{
+		enc := &customEncoder{
 			t:   c.Custom.Type,
 			enc: makeEncoder(c.Custom.Enc.Fn),
 		}
+		if c.Custom.Name != "schema" {
+			return enc
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.

Review comment:
       I don't really understand this comment, or how it relates to the check below.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)

Review comment:
       Looks like another debug println.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -534,26 +1038,34 @@ func (*intervalWindowEncoder) EncodeSingle(elm typex.Window, w io.Writer) error
 
 type intervalWindowDecoder struct{}
 
-func (*intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
+func (d *intervalWindowDecoder) Decode(r io.Reader) ([]typex.Window, error) {
 	// Encoding: upper bound and duration
 
 	n, err := coder.DecodeInt32(r) // #windows
 
 	ret := make([]typex.Window, n, n)
 	for i := int32(0); i < n; i++ {
-		end, err := coder.DecodeEventTime(r)
-		if err != nil {
-			return nil, err
-		}
-		duration, err := coder.DecodeVarUint64(r)
+		w, err := d.DecodeSingle(r)
 		if err != nil {
 			return nil, err
 		}
-		ret[i] = window.IntervalWindow{Start: mtime.FromMilliseconds(end.Milliseconds() - int64(duration)), End: end}
+		ret[i] = w
 	}
 	return ret, err
 }
 
+func (*intervalWindowDecoder) DecodeSingle(r io.Reader) (typex.Window, error) {
+	end, err := coder.DecodeEventTime(r)
+	if err != nil {
+		return nil, err
+	}
+	duration, err := coder.DecodeVarInt(r)

Review comment:
       What's the reason to switch from `DecodeVarUint64` to `DecodeVarInt` here?

##########
File path: sdks/go/test/regression/coders/fromyaml/fromyaml.go
##########
@@ -0,0 +1,415 @@
+// 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.
+
+// fromyaml generates a resource file from the standard_coders.yaml
+// file for use in these coder regression tests.
+//
+// It expects to be run in it's test directory, or via it's go test.
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"math"
+	"reflect"
+	"runtime/debug"
+	"strconv"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/google/go-cmp/cmp"
+	"golang.org/x/text/encoding/charmap"
+	yaml "gopkg.in/yaml.v2"
+)
+
+var unimplementedCoders = map[string]bool{
+	"beam:coder:param_windowed_value:v1": true,
+	"beam:coder:timer:v1":                true,
+}
+
+// Coder is a representation a serialized beam coder.
+type Coder struct {
+	Urn              string  `yaml:"urn,omitempty"`
+	Payload          string  `yaml:"payload,omitempty"`
+	Components       []Coder `yaml:"components,omitempty"`
+	NonDeterministic bool    `yaml:"non_deterministic,omitempty"`
+}
+
+type logger interface {
+	Errorf(string, ...interface{})
+	Logf(string, ...interface{})
+}
+
+// Spec is a set of conditions that a coder must pass.
+type Spec struct {
+	Coder    Coder         `yaml:"coder,omitempty"`
+	Nested   *bool         `yaml:"nested,omitempty"`
+	Examples yaml.MapSlice `yaml:"examples,omitempty"`
+	Log      logger
+
+	id       int // for generating coder ids.
+	coderPBs map[string]*pipepb.Coder
+}
+
+func (s *Spec) nextID() string {
+	ret := fmt.Sprintf("%d", s.id)
+	s.id++
+	return ret
+}
+
+func (s *Spec) testStandardCoder() (err error) {
+	if unimplementedCoders[s.Coder.Urn] {
+		log.Printf("skipping unimplemented coder urn: %v", s.Coder.Urn)
+		return nil
+	}
+	// Construct the coder proto equivalents.
+
+	// Only nested tests need to be run, since nestedness is a pre-portability
+	// concept.
+	// For legacy Java reasons, the row coder examples are all marked nested: false
+	// so we need to check that before skipping unnested tests.
+	if s.Coder.Urn != "beam:coder:row:v1" && s.Nested != nil && !*s.Nested {
+		log.Printf("skipping unnested coder spec: %v\n", s.Coder)
+		return nil
+	}
+
+	s.coderPBs = make(map[string]*pipepb.Coder)
+	id := s.parseCoder(s.Coder)
+	b := graphx.NewCoderUnmarshaller(s.coderPBs)
+	underTest, err := b.Coder(id)
+	if err != nil {
+		return fmt.Errorf("unable to create coder: %v", err)
+	}
+
+	defer func() {
+		if e := recover(); e != nil {
+			err = fmt.Errorf("panicked on coder %v || %v:\n\t%v :\n%s", underTest, s.Coder, e, debug.Stack())
+		}
+	}()
+
+	var decFails, encFails int
+	for _, eg := range s.Examples {
+
+		// Test Decoding
+		// Ideally we'd use the beam package coders, but KVs make that complicated.
+		// This can be cleaned up once a type parametered beam.KV type exists.
+		dec := exec.MakeElementDecoder(underTest)
+		encoded := eg.Key.(string)
+		var elem exec.FullValue
+
+		// What I would have expected.
+		//		r := charmap.ISO8859_1.NewDecoder().Reader(strings.NewReader(encoded))
+		recoded, err := charmap.ISO8859_1.NewEncoder().String(encoded)
+		if err != nil {
+			return err
+		}
+		r := strings.NewReader(recoded)
+		if err := dec.DecodeTo(r, &elem); err != nil {
+			return fmt.Errorf("err decoding %q: %v", encoded, err)
+		}
+		if !diff(s.Coder, &elem, eg) {
+			decFails++
+			continue
+		}
+
+		// Test Encoding
+		if s.Coder.NonDeterministic {
+			// Skip verifying nondeterministic encodings.
+			continue
+		}
+		enc := exec.MakeElementEncoder(underTest)
+		var out bytes.Buffer
+		if err := enc.Encode(&elem, &out); err != nil {
+			return err
+		}
+		if d := cmp.Diff(recoded, string(out.Bytes())); d != "" {
+			log.Printf("Encoding error: diff(-want,+got): %v\n", d)
+		}
+	}
+	if decFails+encFails > 0 {
+		return fmt.Errorf("failed to decode %v times, and encode %v times", decFails, encFails)
+	}
+
+	return nil
+}
+
+var cmpOpts = []cmp.Option{
+	cmp.Transformer("bytes2string", func(in []byte) (out string) {
+		return string(in)
+	}),
+}
+
+func diff(c Coder, elem *exec.FullValue, eg yaml.MapItem) bool {
+	var got, want interface{}
+	switch c.Urn {
+	case "beam:coder:bytes:v1":
+		got = string(elem.Elm.([]byte))
+		switch egv := eg.Value.(type) {
+		case string:
+			want = egv
+		case []byte:
+			want = string(egv)
+		}
+	case "beam:coder:varint:v1":
+		got, want = elem.Elm.(int64), int64(eg.Value.(int))
+	case "beam:coder:double:v1":
+		got = elem.Elm.(float64)
+		switch v := eg.Value.(string); v {
+		case "NaN":
+			// Do the NaN comparison here since NaN by definition != NaN.
+			if math.IsNaN(got.(float64)) {
+				want, got = 1, 1
+			} else {
+				want = math.NaN()
+			}
+		case "-Infinity":
+			want = math.Inf(-1)
+		case "Infinity":
+			want = math.Inf(1)
+		default:
+			want, _ = strconv.ParseFloat(v, 64)
+		}
+
+	case "beam:coder:kv:v1":
+		v := eg.Value.(yaml.MapSlice)
+		pass := true
+		if !diff(c.Components[0], &exec.FullValue{Elm: elem.Elm}, v[0]) {
+			pass = false
+		}
+		if !diff(c.Components[1], &exec.FullValue{Elm: elem.Elm2}, v[1]) {
+			pass = false
+		}
+		return pass
+
+	case "beam:coder:iterable:v1":
+		pass := true
+		gotrv := reflect.ValueOf(elem.Elm)
+		wantrv := reflect.ValueOf(eg.Value)
+		if gotrv.Len() != wantrv.Len() {
+			log.Printf("Lengths don't match. got %v, want %v;  %v, %v", gotrv.Len(), wantrv.Len(), gotrv, wantrv)
+			return false
+		}
+		for i := 0; i < wantrv.Len(); i++ {
+			if !diff(c.Components[0],
+				&exec.FullValue{Elm: gotrv.Index(i).Interface()},
+				yaml.MapItem{Value: wantrv.Index(i).Interface()}) {
+				pass = false
+			}
+
+		}
+		return pass
+	case "beam:coder:interval_window:v1":
+		var a, b int
+		val := eg.Value
+		if is, ok := eg.Value.([]interface{}); ok {
+			val = is[0]
+		}
+		v := val.(yaml.MapSlice)
+
+		a = v[0].Value.(int)
+		b = v[1].Value.(int)
+		end := mtime.FromMilliseconds(int64(a))
+		start := end - mtime.Time(int64(b))
+		want = window.IntervalWindow{Start: start, End: end}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = elem.Elm
+		} else {
+			got = elem.Windows[0]
+		}
+
+	case "beam:coder:global_window:v1":
+		want = window.GlobalWindow{}
+		// If this is nested in an iterable, windows won't be populated.
+		if len(elem.Windows) == 0 {
+			got = window.GlobalWindow(elem.Elm.(struct{}))
+		} else {
+			got = elem.Windows[0]
+		}
+	case "beam:coder:windowed_value:v1", "beam:coder:param_windowed_value:v1":
+		// elem contains all the information, but we need to compare the element+timestamp
+		// separately from the windows, to avoid repeated expected value parsing logic.
+		pass := true
+		vs := eg.Value.(yaml.MapSlice)
+		if !diff(c.Components[0], elem, vs[0]) {
+			pass = false
+		}
+		if d := cmp.Diff(
+			mtime.FromMilliseconds(int64(vs[1].Value.(int))),
+			elem.Timestamp, cmpOpts...); d != "" {
+
+			pass = false
+		}
+		if !diff(c.Components[1], elem, vs[3]) {
+			pass = false
+		}
+		// TODO compare pane information.
+		return pass
+	case "beam:coder:row:v1":
+		fs := eg.Value.(yaml.MapSlice)
+		var rfs []reflect.StructField
+		// There are only 2 pointer examples, but they reuse field names,
+		// so we key off the proto hash to know which example we're handling.
+		ptrEg := strings.Contains(c.Payload, "51ace21c7393")
+		for _, rf := range fs {
+			name := rf.Key.(string)
+			t := nameToType[name]
+			if ptrEg {
+				t = reflect.PtrTo(t)
+			}
+			rfs = append(rfs, reflect.StructField{
+				Name: strings.ToUpper(name[:1]) + name[1:],
+				Type: t,
+				Tag:  reflect.StructTag(fmt.Sprintf("beam:\"%v\"", name)),
+			})
+		}
+		rv := reflect.New(reflect.StructOf(rfs)).Elem()
+		for i, rf := range fs {
+			setField(rv, i, rf.Value)
+		}
+
+		got, want = elem.Elm, rv.Interface()
+	default:
+		got, want = elem.Elm, eg.Value
+	}
+	if d := cmp.Diff(want, got, cmpOpts...); d != "" {
+		log.Printf("Decoding error: diff(-want,+got): %v\n", d)
+		return false
+	}
+	return true
+}
+
+// standard_coders.yaml uses the name for type indication, except for nullability.
+var nameToType = map[string]reflect.Type{
+	"str":     reflectx.String,
+	"i32":     reflectx.Int32,
+	"f64":     reflectx.Float64,
+	"arr":     reflect.SliceOf(reflectx.String),
+	"f_bool":  reflectx.Bool,
+	"f_bytes": reflect.PtrTo(reflectx.ByteSlice),
+	"f_map":   reflect.MapOf(reflectx.String, reflect.PtrTo(reflectx.Int64)),
+}
+
+func setField(rv reflect.Value, i int, v interface{}) {
+	if v == nil {
+		return
+	}
+	rf := rv.Field(i)
+	if rf.Kind() == reflect.Ptr {
+		// Ensure it's initialized.
+		rf.Set(reflect.New(rf.Type().Elem()))
+		rf = rf.Elem()
+	}
+	switch rf.Kind() {
+	case reflect.String:
+		rf.SetString(v.(string))
+	case reflect.Int32:
+		rf.SetInt(int64(v.(int)))
+	case reflect.Float64:
+		c, err := strconv.ParseFloat(v.(string), 64)
+		if err != nil {
+			panic(err)
+		}
+		rf.SetFloat(c)
+	case reflect.Slice:
+		if rf.Type() == reflectx.ByteSlice {
+			rf.Set(reflect.ValueOf([]byte(v.(string))))
+			break
+		}
+		// Value is a []interface{} with string values.

Review comment:
       Is this assumption made specifically based on the yaml file this is meant to be used with? Or is this based on something in the schema spec I'm forgetting?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -117,17 +176,88 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder {
 		return &stringDecoder{}
 
 	case coder.Custom:
-		return &customDecoder{
+		dec := &customDecoder{
 			t:   c.Custom.Type,
 			dec: makeDecoder(c.Custom.Dec.Fn),
 		}
 
+		fmt.Println("getting decoder", c.Custom)
+		if c.Custom.Name != "schema" {
+			return dec
+		}
+		// Custom schema coding is shorthand for using beam infrastructure
+		// wrapped in a custom coder.
+		switch c.T.Type().Kind() {
+		case reflect.Slice:
+			return &lpDecoder{
+				dec: &iterableDecoder{
+					t:   c.Custom.Type,
+					dec: dec,
+				},
+			}
+		case reflect.Array:
+			return &lpDecoder{
+				dec: &arrayDecoder{

Review comment:
       Just to confirm, using an `arrayDecoder` without an `arrayEncoder` specifically works because the format that `iterableEncoder` encodes to is also decodable as an array, right? That might be worth mentioning as a comment, because I was wondering why you can decode arrays with an `arrayDecoder` if they were encoded with an `iterableEncoder`.

##########
File path: sdks/go/pkg/beam/core/runtime/graphx/coder.go
##########
@@ -249,31 +256,35 @@ func (b *CoderUnmarshaller) makeCoder(c *pipepb.Coder) (*coder.Coder, error) {
 			return nil, err
 		}
 
-		// No payload means this coder was length prefixed by the runner
-		// but is likely self describing - AKA a beam coder.
-		if len(sub.GetSpec().GetPayload()) == 0 {
-			return b.makeCoder(sub)
-		}
 		// TODO(lostluck) 2018/10/17: Make this strict again, once dataflow can use
 		// the portable pipeline model directly (BEAM-2885)
-		if sub.GetSpec().GetUrn() != "" && sub.GetSpec().GetUrn() != urnCustomCoder {
-			// TODO(herohde) 11/17/2017: revisit this restriction
-			return nil, errors.Errorf("could not unmarshal length prefix coder from %v, want a custom coder as a sub component but got %v", c, sub)
-		}
-
-		var ref v1pb.CustomCoder
-		if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
-			return nil, err
-		}
-		custom, err := decodeCustomCoder(&ref)
-		if err != nil {
-			return nil, err
+		switch u := sub.GetSpec().GetUrn(); u {
+		case "", urnCustomCoder:
+			var ref v1pb.CustomCoder
+			if err := protox.DecodeBase64(string(sub.GetSpec().GetPayload()), &ref); err != nil {
+				return nil, err
+			}
+			custom, err := decodeCustomCoder(&ref)
+			if err != nil {
+				return nil, err
+			}
+			custom.ID = components[0]
+			t := typex.New(custom.Type)
+			cc := &coder.Coder{Kind: coder.Custom, T: t, Custom: custom}
+			fmt.Println("decoded customcoder", cc)

Review comment:
       Looks like a debug Println left in.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)

Review comment:
       Is it necessary to cast n to an int here? Isn't it already an int from line 638?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {
+		err := c.dec.DecodeTo(r, &e)
+		if err != nil {
+			return reflect.Value{}, err
+		}
+		if e.Elm != nil {
+			rv.Index(i).Set(reflect.ValueOf(e.Elm))
+		} else {
+			rv.Index(i).Set(reflect.ValueOf(e.Windows[0]))

Review comment:
       I don't get why it's setting the value of the element to `e.Windows[0]`. Is it a way to pass in nil?

##########
File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
##########
@@ -434,6 +600,267 @@ func convertIfNeeded(v interface{}, allocated *FullValue) *FullValue {
 	return allocated
 }
 
+type iterableEncoder struct {
+	t   reflect.Type
+	enc ElementEncoder
+}
+
+func (c *iterableEncoder) Encode(val *FullValue, w io.Writer) error {
+	// Do a reflect, get the length.
+	rv := reflect.ValueOf(val.Elm)
+	size := rv.Len()
+	if err := coder.EncodeInt32((int32)(size), w); err != nil {
+		return err
+	}
+	var e FullValue
+	for i := 0; i < size; i++ {
+		e.Elm = rv.Index(i).Interface()
+		err := c.enc.Encode(&e, w)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type iterableDecoder struct {
+	t   reflect.Type
+	dec ElementDecoder
+}
+
+func (c *iterableDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
+	// (1) Read count prefixed encoded data
+
+	size, err := coder.DecodeInt32(r)
+	if err != nil {
+		return err
+	}
+	n := int(size)
+	switch {
+	case n >= 0:
+		rv, err := c.decodeToSlice(int(n), r)
+		if err != nil {
+			return err
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+		return nil
+	case n == -1:
+		rv := reflect.MakeSlice(c.t, 0, 0)
+		chunk, err := coder.DecodeVarInt(r)
+		if err != nil {
+			return err
+		}
+		for chunk != 0 {
+			rvi, err := c.decodeToSlice(int(chunk), r)
+			if err != nil {
+				return err
+			}
+			rv = reflect.AppendSlice(rv, rvi)
+			chunk, err = coder.DecodeVarInt(r)
+			if err != nil {
+				return err
+			}
+		}
+		*fv = FullValue{Elm: rv.Interface()}
+	}
+
+	return nil
+}
+
+func (c *iterableDecoder) decodeToSlice(n int, r io.Reader) (reflect.Value, error) {
+	var e FullValue
+	rv := reflect.MakeSlice(c.t, n, n)
+	for i := 0; i < int(n); i++ {

Review comment:
       Likewise, is it necessary to cast n here too?




----------------------------------------------------------------
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:
users@infra.apache.org