You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/01/02 20:41:00 UTC

[jira] [Commented] (BEAM-3388) Reduce Go runtime reflective overhead

    [ https://issues.apache.org/jira/browse/BEAM-3388?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16308674#comment-16308674 ] 

ASF GitHub Bot commented on BEAM-3388:
--------------------------------------

herohde closed pull request #4323: [BEAM-3388] Type-specialize custom decoders and encoders in Go SDK runtime
URL: https://github.com/apache/beam/pull/4323
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/sdks/go/pkg/beam/core/runtime/exec/coder.go b/sdks/go/pkg/beam/core/runtime/exec/coder.go
index 2253c363016..a96e403aa65 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/coder.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/coder.go
@@ -22,7 +22,6 @@ import (
 	"reflect"
 	"time"
 
-	"github.com/apache/beam/sdks/go/pkg/beam/core/funcx"
 	"github.com/apache/beam/sdks/go/pkg/beam/core/graph"
 	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
 	"github.com/apache/beam/sdks/go/pkg/beam/core/typex"
@@ -89,21 +88,10 @@ func EncodeElement(c *coder.Coder, val FullValue, w io.Writer) error {
 
 		// (1) Call encode
 
-		args := make([]reflect.Value, len(enc.Param))
-		if index, ok := enc.Type(); ok {
-			args[index] = reflect.ValueOf(c.Custom.Type)
-		}
-		params := enc.Params(funcx.FnValue)
-		args[params[0]] = val.Elm
-
-		ret, err := reflectCallNoPanic(enc.Fn, args)
+		data, err := makeEncoder(enc).Encode(c.Custom.Type, val.Elm)
 		if err != nil {
 			return err
 		}
-		if index, ok := enc.Error(); ok && !ret[index].IsNil() {
-			return fmt.Errorf("encode error: %v", ret[index].Interface())
-		}
-		data := ret[enc.Returns(funcx.RetValue)[0]].Interface().([]byte)
 
 		// (2) Add length prefix
 
@@ -166,21 +154,11 @@ func DecodeElement(c *coder.Coder, r io.Reader) (FullValue, error) {
 
 		// (2) Call decode
 
-		args := make([]reflect.Value, len(dec.Param))
-		if index, ok := dec.Type(); ok {
-			args[index] = reflect.ValueOf(c.Custom.Type)
-		}
-		params := dec.Params(funcx.FnValue)
-		args[params[0]] = reflect.ValueOf(data)
-
-		ret, err := reflectCallNoPanic(dec.Fn, args)
+		val, err := makeDecoder(dec).Decode(c.Custom.Type, data)
 		if err != nil {
 			return FullValue{}, err
 		}
-		if index, ok := dec.Error(); ok && !ret[index].IsNil() {
-			return FullValue{}, fmt.Errorf("decode error: %v", ret[index].Interface())
-		}
-		return FullValue{Elm: ret[dec.Returns(funcx.RetValue)[0]]}, err
+		return FullValue{Elm: val}, err
 
 	case coder.KV:
 		key, err := DecodeElement(c.Components[0], r)
diff --git a/sdks/go/pkg/beam/core/runtime/exec/decode.go b/sdks/go/pkg/beam/core/runtime/exec/decode.go
new file mode 100644
index 00000000000..023edc750ed
--- /dev/null
+++ b/sdks/go/pkg/beam/core/runtime/exec/decode.go
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package exec
+
+import (
+	"fmt"
+	"reflect"
+	"sync"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/funcx"
+)
+
+//go:generate specialize --input=decoders.tmpl --x=data,universals
+
+// Decoder is a uniform custom encoder interface.
+type Decoder interface {
+	// Decode decodes the []byte in to a value of the given type.
+	Decode(reflect.Type, []byte) (reflect.Value, error)
+}
+
+var (
+	decoders   = make(map[string]func(reflect.Value) Decoder)
+	decodersMu sync.Mutex
+)
+
+// RegisterDecoder registers an custom decoder invoker for the given type,
+// such as "func(int)[]byte". If multiple decoder invokers are registered
+// for the same type, the last registration wins.
+func RegisterDecoder(t reflect.Type, maker func(reflect.Value) Decoder) {
+	decodersMu.Lock()
+	defer decodersMu.Unlock()
+
+	decoders[t.String()] = maker
+}
+
+func makeDecoder(fn *funcx.Fn) Decoder {
+	decodersMu.Lock()
+	maker, exists := decoders[fn.Fn.Type().String()]
+	decodersMu.Unlock()
+
+	if exists {
+		return maker(fn.Fn)
+	}
+
+	// If no specialized implementation is available, we use the (slower)
+	// reflection-based one.
+
+	return &decoder{fn: fn}
+}
+
+type decoder struct {
+	fn *funcx.Fn
+}
+
+func (d *decoder) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	args := make([]reflect.Value, len(d.fn.Param))
+	if index, ok := d.fn.Type(); ok {
+		args[index] = reflect.ValueOf(t)
+	}
+	params := d.fn.Params(funcx.FnValue)
+	args[params[0]] = reflect.ValueOf(data)
+
+	ret, err := reflectCallNoPanic(d.fn.Fn, args)
+	if err != nil {
+		return reflect.Value{}, err
+	}
+	if index, ok := d.fn.Error(); ok && !ret[index].IsNil() {
+		return reflect.Value{}, fmt.Errorf("decode error: %v", ret[index].Interface())
+	}
+	return ret[d.fn.Returns(funcx.RetValue)[0]], nil
+}
diff --git a/sdks/go/pkg/beam/core/runtime/exec/decoders.go b/sdks/go/pkg/beam/core/runtime/exec/decoders.go
new file mode 100644
index 00000000000..6f191c73ae1
--- /dev/null
+++ b/sdks/go/pkg/beam/core/runtime/exec/decoders.go
@@ -0,0 +1,1347 @@
+// File generated by specialize. Do not edit.
+
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package exec
+
+import (
+	"reflect"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/typex"
+)
+
+func init() {
+	RegisterDecoder(reflect.TypeOf((*func([]byte) []byte)(nil)).Elem(), decoderMakerByteSlice)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) ([]byte, error))(nil)).Elem(), decoderMakerByteSliceE)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) []byte)(nil)).Elem(), decoderMakerTByteSlice)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) ([]byte, error))(nil)).Elem(), decoderMakerTByteSliceE)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) bool)(nil)).Elem(), decoderMakerBool)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (bool, error))(nil)).Elem(), decoderMakerBoolE)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) bool)(nil)).Elem(), decoderMakerTBool)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (bool, error))(nil)).Elem(), decoderMakerTBoolE)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) string)(nil)).Elem(), decoderMakerString)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (string, error))(nil)).Elem(), decoderMakerStringE)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) string)(nil)).Elem(), decoderMakerTString)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (string, error))(nil)).Elem(), decoderMakerTStringE)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) int)(nil)).Elem(), decoderMakerInt)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (int, error))(nil)).Elem(), decoderMakerIntE)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) int)(nil)).Elem(), decoderMakerTInt)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (int, error))(nil)).Elem(), decoderMakerTIntE)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) int8)(nil)).Elem(), decoderMakerInt8)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (int8, error))(nil)).Elem(), decoderMakerInt8E)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) int8)(nil)).Elem(), decoderMakerTInt8)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (int8, error))(nil)).Elem(), decoderMakerTInt8E)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) int16)(nil)).Elem(), decoderMakerInt16)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (int16, error))(nil)).Elem(), decoderMakerInt16E)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) int16)(nil)).Elem(), decoderMakerTInt16)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (int16, error))(nil)).Elem(), decoderMakerTInt16E)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) int32)(nil)).Elem(), decoderMakerInt32)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (int32, error))(nil)).Elem(), decoderMakerInt32E)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) int32)(nil)).Elem(), decoderMakerTInt32)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (int32, error))(nil)).Elem(), decoderMakerTInt32E)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) int64)(nil)).Elem(), decoderMakerInt64)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (int64, error))(nil)).Elem(), decoderMakerInt64E)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) int64)(nil)).Elem(), decoderMakerTInt64)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (int64, error))(nil)).Elem(), decoderMakerTInt64E)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) uint)(nil)).Elem(), decoderMakerUint)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (uint, error))(nil)).Elem(), decoderMakerUintE)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) uint)(nil)).Elem(), decoderMakerTUint)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (uint, error))(nil)).Elem(), decoderMakerTUintE)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) uint8)(nil)).Elem(), decoderMakerUint8)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (uint8, error))(nil)).Elem(), decoderMakerUint8E)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) uint8)(nil)).Elem(), decoderMakerTUint8)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (uint8, error))(nil)).Elem(), decoderMakerTUint8E)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) uint16)(nil)).Elem(), decoderMakerUint16)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (uint16, error))(nil)).Elem(), decoderMakerUint16E)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) uint16)(nil)).Elem(), decoderMakerTUint16)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (uint16, error))(nil)).Elem(), decoderMakerTUint16E)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) uint32)(nil)).Elem(), decoderMakerUint32)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (uint32, error))(nil)).Elem(), decoderMakerUint32E)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) uint32)(nil)).Elem(), decoderMakerTUint32)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (uint32, error))(nil)).Elem(), decoderMakerTUint32E)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) uint64)(nil)).Elem(), decoderMakerUint64)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (uint64, error))(nil)).Elem(), decoderMakerUint64E)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) uint64)(nil)).Elem(), decoderMakerTUint64)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (uint64, error))(nil)).Elem(), decoderMakerTUint64E)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) float32)(nil)).Elem(), decoderMakerFloat32)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (float32, error))(nil)).Elem(), decoderMakerFloat32E)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) float32)(nil)).Elem(), decoderMakerTFloat32)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (float32, error))(nil)).Elem(), decoderMakerTFloat32E)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) float64)(nil)).Elem(), decoderMakerFloat64)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (float64, error))(nil)).Elem(), decoderMakerFloat64E)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) float64)(nil)).Elem(), decoderMakerTFloat64)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (float64, error))(nil)).Elem(), decoderMakerTFloat64E)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) typex.T)(nil)).Elem(), decoderMakerTypex_T)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (typex.T, error))(nil)).Elem(), decoderMakerTypex_TE)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) typex.T)(nil)).Elem(), decoderMakerTTypex_T)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (typex.T, error))(nil)).Elem(), decoderMakerTTypex_TE)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) typex.U)(nil)).Elem(), decoderMakerTypex_U)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (typex.U, error))(nil)).Elem(), decoderMakerTypex_UE)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) typex.U)(nil)).Elem(), decoderMakerTTypex_U)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (typex.U, error))(nil)).Elem(), decoderMakerTTypex_UE)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) typex.V)(nil)).Elem(), decoderMakerTypex_V)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (typex.V, error))(nil)).Elem(), decoderMakerTypex_VE)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) typex.V)(nil)).Elem(), decoderMakerTTypex_V)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (typex.V, error))(nil)).Elem(), decoderMakerTTypex_VE)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) typex.W)(nil)).Elem(), decoderMakerTypex_W)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (typex.W, error))(nil)).Elem(), decoderMakerTypex_WE)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) typex.W)(nil)).Elem(), decoderMakerTTypex_W)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (typex.W, error))(nil)).Elem(), decoderMakerTTypex_WE)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) typex.X)(nil)).Elem(), decoderMakerTypex_X)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (typex.X, error))(nil)).Elem(), decoderMakerTypex_XE)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) typex.X)(nil)).Elem(), decoderMakerTTypex_X)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (typex.X, error))(nil)).Elem(), decoderMakerTTypex_XE)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) typex.Y)(nil)).Elem(), decoderMakerTypex_Y)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (typex.Y, error))(nil)).Elem(), decoderMakerTypex_YE)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) typex.Y)(nil)).Elem(), decoderMakerTTypex_Y)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (typex.Y, error))(nil)).Elem(), decoderMakerTTypex_YE)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) typex.Z)(nil)).Elem(), decoderMakerTypex_Z)
+	RegisterDecoder(reflect.TypeOf((*func([]byte) (typex.Z, error))(nil)).Elem(), decoderMakerTypex_ZE)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) typex.Z)(nil)).Elem(), decoderMakerTTypex_Z)
+	RegisterDecoder(reflect.TypeOf((*func(reflect.Type, []byte) (typex.Z, error))(nil)).Elem(), decoderMakerTTypex_ZE)
+}
+
+type decoderByteSlice struct {
+	fn func([]byte) []byte
+}
+
+func (e *decoderByteSlice) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerByteSlice(fn reflect.Value) Decoder {
+	return &decoderByteSlice{fn: fn.Interface().(func([]byte) []byte)}
+}
+
+type decoderByteSliceE struct {
+	fn func([]byte) ([]byte, error)
+}
+
+func (e *decoderByteSliceE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerByteSliceE(fn reflect.Value) Decoder {
+	return &decoderByteSliceE{fn: fn.Interface().(func([]byte) ([]byte, error))}
+}
+
+type decoderTByteSlice struct {
+	fn func(reflect.Type, []byte) []byte
+}
+
+func (e *decoderTByteSlice) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTByteSlice(fn reflect.Value) Decoder {
+	return &decoderTByteSlice{fn: fn.Interface().(func(reflect.Type, []byte) []byte)}
+}
+
+type decoderTByteSliceE struct {
+	fn func(reflect.Type, []byte) ([]byte, error)
+}
+
+func (e *decoderTByteSliceE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTByteSliceE(fn reflect.Value) Decoder {
+	return &decoderTByteSliceE{fn: fn.Interface().(func(reflect.Type, []byte) ([]byte, error))}
+}
+
+type decoderBool struct {
+	fn func([]byte) bool
+}
+
+func (e *decoderBool) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerBool(fn reflect.Value) Decoder {
+	return &decoderBool{fn: fn.Interface().(func([]byte) bool)}
+}
+
+type decoderBoolE struct {
+	fn func([]byte) (bool, error)
+}
+
+func (e *decoderBoolE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerBoolE(fn reflect.Value) Decoder {
+	return &decoderBoolE{fn: fn.Interface().(func([]byte) (bool, error))}
+}
+
+type decoderTBool struct {
+	fn func(reflect.Type, []byte) bool
+}
+
+func (e *decoderTBool) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTBool(fn reflect.Value) Decoder {
+	return &decoderTBool{fn: fn.Interface().(func(reflect.Type, []byte) bool)}
+}
+
+type decoderTBoolE struct {
+	fn func(reflect.Type, []byte) (bool, error)
+}
+
+func (e *decoderTBoolE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTBoolE(fn reflect.Value) Decoder {
+	return &decoderTBoolE{fn: fn.Interface().(func(reflect.Type, []byte) (bool, error))}
+}
+
+type decoderString struct {
+	fn func([]byte) string
+}
+
+func (e *decoderString) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerString(fn reflect.Value) Decoder {
+	return &decoderString{fn: fn.Interface().(func([]byte) string)}
+}
+
+type decoderStringE struct {
+	fn func([]byte) (string, error)
+}
+
+func (e *decoderStringE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerStringE(fn reflect.Value) Decoder {
+	return &decoderStringE{fn: fn.Interface().(func([]byte) (string, error))}
+}
+
+type decoderTString struct {
+	fn func(reflect.Type, []byte) string
+}
+
+func (e *decoderTString) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTString(fn reflect.Value) Decoder {
+	return &decoderTString{fn: fn.Interface().(func(reflect.Type, []byte) string)}
+}
+
+type decoderTStringE struct {
+	fn func(reflect.Type, []byte) (string, error)
+}
+
+func (e *decoderTStringE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTStringE(fn reflect.Value) Decoder {
+	return &decoderTStringE{fn: fn.Interface().(func(reflect.Type, []byte) (string, error))}
+}
+
+type decoderInt struct {
+	fn func([]byte) int
+}
+
+func (e *decoderInt) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerInt(fn reflect.Value) Decoder {
+	return &decoderInt{fn: fn.Interface().(func([]byte) int)}
+}
+
+type decoderIntE struct {
+	fn func([]byte) (int, error)
+}
+
+func (e *decoderIntE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerIntE(fn reflect.Value) Decoder {
+	return &decoderIntE{fn: fn.Interface().(func([]byte) (int, error))}
+}
+
+type decoderTInt struct {
+	fn func(reflect.Type, []byte) int
+}
+
+func (e *decoderTInt) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTInt(fn reflect.Value) Decoder {
+	return &decoderTInt{fn: fn.Interface().(func(reflect.Type, []byte) int)}
+}
+
+type decoderTIntE struct {
+	fn func(reflect.Type, []byte) (int, error)
+}
+
+func (e *decoderTIntE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTIntE(fn reflect.Value) Decoder {
+	return &decoderTIntE{fn: fn.Interface().(func(reflect.Type, []byte) (int, error))}
+}
+
+type decoderInt8 struct {
+	fn func([]byte) int8
+}
+
+func (e *decoderInt8) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerInt8(fn reflect.Value) Decoder {
+	return &decoderInt8{fn: fn.Interface().(func([]byte) int8)}
+}
+
+type decoderInt8E struct {
+	fn func([]byte) (int8, error)
+}
+
+func (e *decoderInt8E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerInt8E(fn reflect.Value) Decoder {
+	return &decoderInt8E{fn: fn.Interface().(func([]byte) (int8, error))}
+}
+
+type decoderTInt8 struct {
+	fn func(reflect.Type, []byte) int8
+}
+
+func (e *decoderTInt8) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTInt8(fn reflect.Value) Decoder {
+	return &decoderTInt8{fn: fn.Interface().(func(reflect.Type, []byte) int8)}
+}
+
+type decoderTInt8E struct {
+	fn func(reflect.Type, []byte) (int8, error)
+}
+
+func (e *decoderTInt8E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTInt8E(fn reflect.Value) Decoder {
+	return &decoderTInt8E{fn: fn.Interface().(func(reflect.Type, []byte) (int8, error))}
+}
+
+type decoderInt16 struct {
+	fn func([]byte) int16
+}
+
+func (e *decoderInt16) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerInt16(fn reflect.Value) Decoder {
+	return &decoderInt16{fn: fn.Interface().(func([]byte) int16)}
+}
+
+type decoderInt16E struct {
+	fn func([]byte) (int16, error)
+}
+
+func (e *decoderInt16E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerInt16E(fn reflect.Value) Decoder {
+	return &decoderInt16E{fn: fn.Interface().(func([]byte) (int16, error))}
+}
+
+type decoderTInt16 struct {
+	fn func(reflect.Type, []byte) int16
+}
+
+func (e *decoderTInt16) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTInt16(fn reflect.Value) Decoder {
+	return &decoderTInt16{fn: fn.Interface().(func(reflect.Type, []byte) int16)}
+}
+
+type decoderTInt16E struct {
+	fn func(reflect.Type, []byte) (int16, error)
+}
+
+func (e *decoderTInt16E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTInt16E(fn reflect.Value) Decoder {
+	return &decoderTInt16E{fn: fn.Interface().(func(reflect.Type, []byte) (int16, error))}
+}
+
+type decoderInt32 struct {
+	fn func([]byte) int32
+}
+
+func (e *decoderInt32) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerInt32(fn reflect.Value) Decoder {
+	return &decoderInt32{fn: fn.Interface().(func([]byte) int32)}
+}
+
+type decoderInt32E struct {
+	fn func([]byte) (int32, error)
+}
+
+func (e *decoderInt32E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerInt32E(fn reflect.Value) Decoder {
+	return &decoderInt32E{fn: fn.Interface().(func([]byte) (int32, error))}
+}
+
+type decoderTInt32 struct {
+	fn func(reflect.Type, []byte) int32
+}
+
+func (e *decoderTInt32) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTInt32(fn reflect.Value) Decoder {
+	return &decoderTInt32{fn: fn.Interface().(func(reflect.Type, []byte) int32)}
+}
+
+type decoderTInt32E struct {
+	fn func(reflect.Type, []byte) (int32, error)
+}
+
+func (e *decoderTInt32E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTInt32E(fn reflect.Value) Decoder {
+	return &decoderTInt32E{fn: fn.Interface().(func(reflect.Type, []byte) (int32, error))}
+}
+
+type decoderInt64 struct {
+	fn func([]byte) int64
+}
+
+func (e *decoderInt64) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerInt64(fn reflect.Value) Decoder {
+	return &decoderInt64{fn: fn.Interface().(func([]byte) int64)}
+}
+
+type decoderInt64E struct {
+	fn func([]byte) (int64, error)
+}
+
+func (e *decoderInt64E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerInt64E(fn reflect.Value) Decoder {
+	return &decoderInt64E{fn: fn.Interface().(func([]byte) (int64, error))}
+}
+
+type decoderTInt64 struct {
+	fn func(reflect.Type, []byte) int64
+}
+
+func (e *decoderTInt64) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTInt64(fn reflect.Value) Decoder {
+	return &decoderTInt64{fn: fn.Interface().(func(reflect.Type, []byte) int64)}
+}
+
+type decoderTInt64E struct {
+	fn func(reflect.Type, []byte) (int64, error)
+}
+
+func (e *decoderTInt64E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTInt64E(fn reflect.Value) Decoder {
+	return &decoderTInt64E{fn: fn.Interface().(func(reflect.Type, []byte) (int64, error))}
+}
+
+type decoderUint struct {
+	fn func([]byte) uint
+}
+
+func (e *decoderUint) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerUint(fn reflect.Value) Decoder {
+	return &decoderUint{fn: fn.Interface().(func([]byte) uint)}
+}
+
+type decoderUintE struct {
+	fn func([]byte) (uint, error)
+}
+
+func (e *decoderUintE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerUintE(fn reflect.Value) Decoder {
+	return &decoderUintE{fn: fn.Interface().(func([]byte) (uint, error))}
+}
+
+type decoderTUint struct {
+	fn func(reflect.Type, []byte) uint
+}
+
+func (e *decoderTUint) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTUint(fn reflect.Value) Decoder {
+	return &decoderTUint{fn: fn.Interface().(func(reflect.Type, []byte) uint)}
+}
+
+type decoderTUintE struct {
+	fn func(reflect.Type, []byte) (uint, error)
+}
+
+func (e *decoderTUintE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTUintE(fn reflect.Value) Decoder {
+	return &decoderTUintE{fn: fn.Interface().(func(reflect.Type, []byte) (uint, error))}
+}
+
+type decoderUint8 struct {
+	fn func([]byte) uint8
+}
+
+func (e *decoderUint8) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerUint8(fn reflect.Value) Decoder {
+	return &decoderUint8{fn: fn.Interface().(func([]byte) uint8)}
+}
+
+type decoderUint8E struct {
+	fn func([]byte) (uint8, error)
+}
+
+func (e *decoderUint8E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerUint8E(fn reflect.Value) Decoder {
+	return &decoderUint8E{fn: fn.Interface().(func([]byte) (uint8, error))}
+}
+
+type decoderTUint8 struct {
+	fn func(reflect.Type, []byte) uint8
+}
+
+func (e *decoderTUint8) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTUint8(fn reflect.Value) Decoder {
+	return &decoderTUint8{fn: fn.Interface().(func(reflect.Type, []byte) uint8)}
+}
+
+type decoderTUint8E struct {
+	fn func(reflect.Type, []byte) (uint8, error)
+}
+
+func (e *decoderTUint8E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTUint8E(fn reflect.Value) Decoder {
+	return &decoderTUint8E{fn: fn.Interface().(func(reflect.Type, []byte) (uint8, error))}
+}
+
+type decoderUint16 struct {
+	fn func([]byte) uint16
+}
+
+func (e *decoderUint16) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerUint16(fn reflect.Value) Decoder {
+	return &decoderUint16{fn: fn.Interface().(func([]byte) uint16)}
+}
+
+type decoderUint16E struct {
+	fn func([]byte) (uint16, error)
+}
+
+func (e *decoderUint16E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerUint16E(fn reflect.Value) Decoder {
+	return &decoderUint16E{fn: fn.Interface().(func([]byte) (uint16, error))}
+}
+
+type decoderTUint16 struct {
+	fn func(reflect.Type, []byte) uint16
+}
+
+func (e *decoderTUint16) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTUint16(fn reflect.Value) Decoder {
+	return &decoderTUint16{fn: fn.Interface().(func(reflect.Type, []byte) uint16)}
+}
+
+type decoderTUint16E struct {
+	fn func(reflect.Type, []byte) (uint16, error)
+}
+
+func (e *decoderTUint16E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTUint16E(fn reflect.Value) Decoder {
+	return &decoderTUint16E{fn: fn.Interface().(func(reflect.Type, []byte) (uint16, error))}
+}
+
+type decoderUint32 struct {
+	fn func([]byte) uint32
+}
+
+func (e *decoderUint32) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerUint32(fn reflect.Value) Decoder {
+	return &decoderUint32{fn: fn.Interface().(func([]byte) uint32)}
+}
+
+type decoderUint32E struct {
+	fn func([]byte) (uint32, error)
+}
+
+func (e *decoderUint32E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerUint32E(fn reflect.Value) Decoder {
+	return &decoderUint32E{fn: fn.Interface().(func([]byte) (uint32, error))}
+}
+
+type decoderTUint32 struct {
+	fn func(reflect.Type, []byte) uint32
+}
+
+func (e *decoderTUint32) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTUint32(fn reflect.Value) Decoder {
+	return &decoderTUint32{fn: fn.Interface().(func(reflect.Type, []byte) uint32)}
+}
+
+type decoderTUint32E struct {
+	fn func(reflect.Type, []byte) (uint32, error)
+}
+
+func (e *decoderTUint32E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTUint32E(fn reflect.Value) Decoder {
+	return &decoderTUint32E{fn: fn.Interface().(func(reflect.Type, []byte) (uint32, error))}
+}
+
+type decoderUint64 struct {
+	fn func([]byte) uint64
+}
+
+func (e *decoderUint64) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerUint64(fn reflect.Value) Decoder {
+	return &decoderUint64{fn: fn.Interface().(func([]byte) uint64)}
+}
+
+type decoderUint64E struct {
+	fn func([]byte) (uint64, error)
+}
+
+func (e *decoderUint64E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerUint64E(fn reflect.Value) Decoder {
+	return &decoderUint64E{fn: fn.Interface().(func([]byte) (uint64, error))}
+}
+
+type decoderTUint64 struct {
+	fn func(reflect.Type, []byte) uint64
+}
+
+func (e *decoderTUint64) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTUint64(fn reflect.Value) Decoder {
+	return &decoderTUint64{fn: fn.Interface().(func(reflect.Type, []byte) uint64)}
+}
+
+type decoderTUint64E struct {
+	fn func(reflect.Type, []byte) (uint64, error)
+}
+
+func (e *decoderTUint64E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTUint64E(fn reflect.Value) Decoder {
+	return &decoderTUint64E{fn: fn.Interface().(func(reflect.Type, []byte) (uint64, error))}
+}
+
+type decoderFloat32 struct {
+	fn func([]byte) float32
+}
+
+func (e *decoderFloat32) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerFloat32(fn reflect.Value) Decoder {
+	return &decoderFloat32{fn: fn.Interface().(func([]byte) float32)}
+}
+
+type decoderFloat32E struct {
+	fn func([]byte) (float32, error)
+}
+
+func (e *decoderFloat32E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerFloat32E(fn reflect.Value) Decoder {
+	return &decoderFloat32E{fn: fn.Interface().(func([]byte) (float32, error))}
+}
+
+type decoderTFloat32 struct {
+	fn func(reflect.Type, []byte) float32
+}
+
+func (e *decoderTFloat32) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTFloat32(fn reflect.Value) Decoder {
+	return &decoderTFloat32{fn: fn.Interface().(func(reflect.Type, []byte) float32)}
+}
+
+type decoderTFloat32E struct {
+	fn func(reflect.Type, []byte) (float32, error)
+}
+
+func (e *decoderTFloat32E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTFloat32E(fn reflect.Value) Decoder {
+	return &decoderTFloat32E{fn: fn.Interface().(func(reflect.Type, []byte) (float32, error))}
+}
+
+type decoderFloat64 struct {
+	fn func([]byte) float64
+}
+
+func (e *decoderFloat64) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerFloat64(fn reflect.Value) Decoder {
+	return &decoderFloat64{fn: fn.Interface().(func([]byte) float64)}
+}
+
+type decoderFloat64E struct {
+	fn func([]byte) (float64, error)
+}
+
+func (e *decoderFloat64E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerFloat64E(fn reflect.Value) Decoder {
+	return &decoderFloat64E{fn: fn.Interface().(func([]byte) (float64, error))}
+}
+
+type decoderTFloat64 struct {
+	fn func(reflect.Type, []byte) float64
+}
+
+func (e *decoderTFloat64) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTFloat64(fn reflect.Value) Decoder {
+	return &decoderTFloat64{fn: fn.Interface().(func(reflect.Type, []byte) float64)}
+}
+
+type decoderTFloat64E struct {
+	fn func(reflect.Type, []byte) (float64, error)
+}
+
+func (e *decoderTFloat64E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTFloat64E(fn reflect.Value) Decoder {
+	return &decoderTFloat64E{fn: fn.Interface().(func(reflect.Type, []byte) (float64, error))}
+}
+
+type decoderTypex_T struct {
+	fn func([]byte) typex.T
+}
+
+func (e *decoderTypex_T) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerTypex_T(fn reflect.Value) Decoder {
+	return &decoderTypex_T{fn: fn.Interface().(func([]byte) typex.T)}
+}
+
+type decoderTypex_TE struct {
+	fn func([]byte) (typex.T, error)
+}
+
+func (e *decoderTypex_TE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTypex_TE(fn reflect.Value) Decoder {
+	return &decoderTypex_TE{fn: fn.Interface().(func([]byte) (typex.T, error))}
+}
+
+type decoderTTypex_T struct {
+	fn func(reflect.Type, []byte) typex.T
+}
+
+func (e *decoderTTypex_T) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTTypex_T(fn reflect.Value) Decoder {
+	return &decoderTTypex_T{fn: fn.Interface().(func(reflect.Type, []byte) typex.T)}
+}
+
+type decoderTTypex_TE struct {
+	fn func(reflect.Type, []byte) (typex.T, error)
+}
+
+func (e *decoderTTypex_TE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTTypex_TE(fn reflect.Value) Decoder {
+	return &decoderTTypex_TE{fn: fn.Interface().(func(reflect.Type, []byte) (typex.T, error))}
+}
+
+type decoderTypex_U struct {
+	fn func([]byte) typex.U
+}
+
+func (e *decoderTypex_U) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerTypex_U(fn reflect.Value) Decoder {
+	return &decoderTypex_U{fn: fn.Interface().(func([]byte) typex.U)}
+}
+
+type decoderTypex_UE struct {
+	fn func([]byte) (typex.U, error)
+}
+
+func (e *decoderTypex_UE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTypex_UE(fn reflect.Value) Decoder {
+	return &decoderTypex_UE{fn: fn.Interface().(func([]byte) (typex.U, error))}
+}
+
+type decoderTTypex_U struct {
+	fn func(reflect.Type, []byte) typex.U
+}
+
+func (e *decoderTTypex_U) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTTypex_U(fn reflect.Value) Decoder {
+	return &decoderTTypex_U{fn: fn.Interface().(func(reflect.Type, []byte) typex.U)}
+}
+
+type decoderTTypex_UE struct {
+	fn func(reflect.Type, []byte) (typex.U, error)
+}
+
+func (e *decoderTTypex_UE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTTypex_UE(fn reflect.Value) Decoder {
+	return &decoderTTypex_UE{fn: fn.Interface().(func(reflect.Type, []byte) (typex.U, error))}
+}
+
+type decoderTypex_V struct {
+	fn func([]byte) typex.V
+}
+
+func (e *decoderTypex_V) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerTypex_V(fn reflect.Value) Decoder {
+	return &decoderTypex_V{fn: fn.Interface().(func([]byte) typex.V)}
+}
+
+type decoderTypex_VE struct {
+	fn func([]byte) (typex.V, error)
+}
+
+func (e *decoderTypex_VE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTypex_VE(fn reflect.Value) Decoder {
+	return &decoderTypex_VE{fn: fn.Interface().(func([]byte) (typex.V, error))}
+}
+
+type decoderTTypex_V struct {
+	fn func(reflect.Type, []byte) typex.V
+}
+
+func (e *decoderTTypex_V) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTTypex_V(fn reflect.Value) Decoder {
+	return &decoderTTypex_V{fn: fn.Interface().(func(reflect.Type, []byte) typex.V)}
+}
+
+type decoderTTypex_VE struct {
+	fn func(reflect.Type, []byte) (typex.V, error)
+}
+
+func (e *decoderTTypex_VE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTTypex_VE(fn reflect.Value) Decoder {
+	return &decoderTTypex_VE{fn: fn.Interface().(func(reflect.Type, []byte) (typex.V, error))}
+}
+
+type decoderTypex_W struct {
+	fn func([]byte) typex.W
+}
+
+func (e *decoderTypex_W) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerTypex_W(fn reflect.Value) Decoder {
+	return &decoderTypex_W{fn: fn.Interface().(func([]byte) typex.W)}
+}
+
+type decoderTypex_WE struct {
+	fn func([]byte) (typex.W, error)
+}
+
+func (e *decoderTypex_WE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTypex_WE(fn reflect.Value) Decoder {
+	return &decoderTypex_WE{fn: fn.Interface().(func([]byte) (typex.W, error))}
+}
+
+type decoderTTypex_W struct {
+	fn func(reflect.Type, []byte) typex.W
+}
+
+func (e *decoderTTypex_W) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTTypex_W(fn reflect.Value) Decoder {
+	return &decoderTTypex_W{fn: fn.Interface().(func(reflect.Type, []byte) typex.W)}
+}
+
+type decoderTTypex_WE struct {
+	fn func(reflect.Type, []byte) (typex.W, error)
+}
+
+func (e *decoderTTypex_WE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTTypex_WE(fn reflect.Value) Decoder {
+	return &decoderTTypex_WE{fn: fn.Interface().(func(reflect.Type, []byte) (typex.W, error))}
+}
+
+type decoderTypex_X struct {
+	fn func([]byte) typex.X
+}
+
+func (e *decoderTypex_X) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerTypex_X(fn reflect.Value) Decoder {
+	return &decoderTypex_X{fn: fn.Interface().(func([]byte) typex.X)}
+}
+
+type decoderTypex_XE struct {
+	fn func([]byte) (typex.X, error)
+}
+
+func (e *decoderTypex_XE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTypex_XE(fn reflect.Value) Decoder {
+	return &decoderTypex_XE{fn: fn.Interface().(func([]byte) (typex.X, error))}
+}
+
+type decoderTTypex_X struct {
+	fn func(reflect.Type, []byte) typex.X
+}
+
+func (e *decoderTTypex_X) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTTypex_X(fn reflect.Value) Decoder {
+	return &decoderTTypex_X{fn: fn.Interface().(func(reflect.Type, []byte) typex.X)}
+}
+
+type decoderTTypex_XE struct {
+	fn func(reflect.Type, []byte) (typex.X, error)
+}
+
+func (e *decoderTTypex_XE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTTypex_XE(fn reflect.Value) Decoder {
+	return &decoderTTypex_XE{fn: fn.Interface().(func(reflect.Type, []byte) (typex.X, error))}
+}
+
+type decoderTypex_Y struct {
+	fn func([]byte) typex.Y
+}
+
+func (e *decoderTypex_Y) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerTypex_Y(fn reflect.Value) Decoder {
+	return &decoderTypex_Y{fn: fn.Interface().(func([]byte) typex.Y)}
+}
+
+type decoderTypex_YE struct {
+	fn func([]byte) (typex.Y, error)
+}
+
+func (e *decoderTypex_YE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTypex_YE(fn reflect.Value) Decoder {
+	return &decoderTypex_YE{fn: fn.Interface().(func([]byte) (typex.Y, error))}
+}
+
+type decoderTTypex_Y struct {
+	fn func(reflect.Type, []byte) typex.Y
+}
+
+func (e *decoderTTypex_Y) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTTypex_Y(fn reflect.Value) Decoder {
+	return &decoderTTypex_Y{fn: fn.Interface().(func(reflect.Type, []byte) typex.Y)}
+}
+
+type decoderTTypex_YE struct {
+	fn func(reflect.Type, []byte) (typex.Y, error)
+}
+
+func (e *decoderTTypex_YE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTTypex_YE(fn reflect.Value) Decoder {
+	return &decoderTTypex_YE{fn: fn.Interface().(func(reflect.Type, []byte) (typex.Y, error))}
+}
+
+type decoderTypex_Z struct {
+	fn func([]byte) typex.Z
+}
+
+func (e *decoderTypex_Z) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMakerTypex_Z(fn reflect.Value) Decoder {
+	return &decoderTypex_Z{fn: fn.Interface().(func([]byte) typex.Z)}
+}
+
+type decoderTypex_ZE struct {
+	fn func([]byte) (typex.Z, error)
+}
+
+func (e *decoderTypex_ZE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTypex_ZE(fn reflect.Value) Decoder {
+	return &decoderTypex_ZE{fn: fn.Interface().(func([]byte) (typex.Z, error))}
+}
+
+type decoderTTypex_Z struct {
+	fn func(reflect.Type, []byte) typex.Z
+}
+
+func (e *decoderTTypex_Z) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerTTypex_Z(fn reflect.Value) Decoder {
+	return &decoderTTypex_Z{fn: fn.Interface().(func(reflect.Type, []byte) typex.Z)}
+}
+
+type decoderTTypex_ZE struct {
+	fn func(reflect.Type, []byte) (typex.Z, error)
+}
+
+func (e *decoderTTypex_ZE) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+	val, err := e.fn(t, data)
+	if err != nil {
+		return reflect.Value{}, nil
+	}
+	return reflect.ValueOf(val), nil
+}
+
+func decoderMakerTTypex_ZE(fn reflect.Value) Decoder {
+	return &decoderTTypex_ZE{fn: fn.Interface().(func(reflect.Type, []byte) (typex.Z, error))}
+}
diff --git a/sdks/go/pkg/beam/core/runtime/exec/decoders.tmpl b/sdks/go/pkg/beam/core/runtime/exec/decoders.tmpl
new file mode 100644
index 00000000000..1df00c76879
--- /dev/null
+++ b/sdks/go/pkg/beam/core/runtime/exec/decoders.tmpl
@@ -0,0 +1,89 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package exec
+
+import (
+    "reflect"
+
+  	"github.com/apache/beam/sdks/go/pkg/beam/core/typex"
+)
+
+func init() {
+{{- range $x := .X}}
+    RegisterDecoder(reflect.TypeOf((*func ([]byte) {{$x.Type}})(nil)).Elem(), decoderMaker{{$x.Name}})
+    RegisterDecoder(reflect.TypeOf((*func ([]byte) ({{$x.Type}}, error))(nil)).Elem(), decoderMaker{{$x.Name}}E)
+    RegisterDecoder(reflect.TypeOf((*func (reflect.Type, []byte) {{$x.Type}})(nil)).Elem(), decoderMakerT{{$x.Name}})
+    RegisterDecoder(reflect.TypeOf((*func (reflect.Type, []byte) ({{$x.Type}}, error))(nil)).Elem(), decoderMakerT{{$x.Name}}E)
+{{- end}}
+}
+
+{{range $x := .X}}
+type decoder{{$x.Name}} struct {
+    fn func ([]byte) {{$x.Type}}
+}
+
+func (e *decoder{{$x.Name}}) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+    return reflect.ValueOf(e.fn(data)), nil
+}
+
+func decoderMaker{{$x.Name}}(fn reflect.Value) Decoder {
+    return &decoder{{$x.Name}}{fn: fn.Interface().(func ([]byte) {{$x.Type}})}
+}
+
+type decoder{{$x.Name}}E struct {
+    fn func ([]byte) ({{$x.Type}}, error)
+}
+
+func (e *decoder{{$x.Name}}E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+    val, err := e.fn(data)
+    if err != nil {
+        return reflect.Value{}, nil
+    }
+    return reflect.ValueOf(val), nil
+}
+
+func decoderMaker{{$x.Name}}E(fn reflect.Value) Decoder {
+    return &decoder{{$x.Name}}E{fn: fn.Interface().(func ([]byte) ({{$x.Type}}, error))}
+}
+
+type decoderT{{$x.Name}} struct {
+    fn func (reflect.Type, []byte) {{$x.Type}}
+}
+
+func (e *decoderT{{$x.Name}}) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+    return reflect.ValueOf(e.fn(t, data)), nil
+}
+
+func decoderMakerT{{$x.Name}}(fn reflect.Value) Decoder {
+    return &decoderT{{$x.Name}}{fn: fn.Interface().(func (reflect.Type, []byte) {{$x.Type}})}
+}
+
+type decoderT{{$x.Name}}E struct {
+    fn func (reflect.Type, []byte) ({{$x.Type}}, error)
+}
+
+func (e *decoderT{{$x.Name}}E) Decode(t reflect.Type, data []byte) (reflect.Value, error) {
+    val, err := e.fn(t, data)
+    if err != nil {
+        return reflect.Value{}, nil
+    }
+    return reflect.ValueOf(val), nil
+}
+
+func decoderMakerT{{$x.Name}}E(fn reflect.Value) Decoder {
+    return &decoderT{{$x.Name}}E{fn: fn.Interface().(func (reflect.Type, []byte) ({{$x.Type}}, error))}
+}
+{{end}}
diff --git a/sdks/go/pkg/beam/core/runtime/exec/encode.go b/sdks/go/pkg/beam/core/runtime/exec/encode.go
new file mode 100644
index 00000000000..982cee4fec4
--- /dev/null
+++ b/sdks/go/pkg/beam/core/runtime/exec/encode.go
@@ -0,0 +1,85 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package exec
+
+import (
+	"fmt"
+	"reflect"
+	"sync"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/funcx"
+)
+
+//go:generate specialize --input=encoders.tmpl --x=data,universals
+
+// Encoder is a uniform custom encoder interface.
+type Encoder interface {
+	// Encode encodes the given value (of the given type).
+	Encode(reflect.Type, reflect.Value) ([]byte, error)
+}
+
+var (
+	encoders   = make(map[string]func(reflect.Value) Encoder)
+	encodersMu sync.Mutex
+)
+
+// RegisterEncoder registers an custom encoder invoker for the given type,
+// such as "func(int)[]byte". If multiple encoder invokers are registered
+// for the same type, the last registration wins.
+func RegisterEncoder(t reflect.Type, maker func(reflect.Value) Encoder) {
+	encodersMu.Lock()
+	defer encodersMu.Unlock()
+
+	encoders[t.String()] = maker
+}
+
+func makeEncoder(fn *funcx.Fn) Encoder {
+	encodersMu.Lock()
+	maker, exists := encoders[fn.Fn.Type().String()]
+	encodersMu.Unlock()
+
+	if exists {
+		return maker(fn.Fn)
+	}
+
+	// If no specialized implementation is available, we use the (slower)
+	// reflection-based one.
+
+	return &encoder{fn: fn}
+}
+
+type encoder struct {
+	fn *funcx.Fn
+}
+
+func (e *encoder) Encode(t reflect.Type, elm reflect.Value) ([]byte, error) {
+	args := make([]reflect.Value, len(e.fn.Param))
+	if index, ok := e.fn.Type(); ok {
+		args[index] = reflect.ValueOf(t)
+	}
+	params := e.fn.Params(funcx.FnValue)
+	args[params[0]] = elm
+
+	ret, err := reflectCallNoPanic(e.fn.Fn, args)
+	if err != nil {
+		return nil, err
+	}
+	if index, ok := e.fn.Error(); ok && !ret[index].IsNil() {
+		return nil, fmt.Errorf("encode error: %v", ret[index].Interface())
+	}
+	data := ret[e.fn.Returns(funcx.RetValue)[0]].Interface().([]byte)
+	return data, nil
+}
diff --git a/sdks/go/pkg/beam/core/runtime/exec/encoders.go b/sdks/go/pkg/beam/core/runtime/exec/encoders.go
new file mode 100644
index 00000000000..7cdff48d9ed
--- /dev/null
+++ b/sdks/go/pkg/beam/core/runtime/exec/encoders.go
@@ -0,0 +1,1171 @@
+// File generated by specialize. Do not edit.
+
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package exec
+
+import (
+	"reflect"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/typex"
+)
+
+func init() {
+	RegisterEncoder(reflect.TypeOf((*func([]byte) []byte)(nil)).Elem(), encoderMakerByteSlice)
+	RegisterEncoder(reflect.TypeOf((*func([]byte) ([]byte, error))(nil)).Elem(), encoderMakerByteSliceE)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, []byte) []byte)(nil)).Elem(), encoderMakerTByteSlice)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, []byte) ([]byte, error))(nil)).Elem(), encoderMakerTByteSliceE)
+	RegisterEncoder(reflect.TypeOf((*func(bool) []byte)(nil)).Elem(), encoderMakerBool)
+	RegisterEncoder(reflect.TypeOf((*func(bool) ([]byte, error))(nil)).Elem(), encoderMakerBoolE)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, bool) []byte)(nil)).Elem(), encoderMakerTBool)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, bool) ([]byte, error))(nil)).Elem(), encoderMakerTBoolE)
+	RegisterEncoder(reflect.TypeOf((*func(string) []byte)(nil)).Elem(), encoderMakerString)
+	RegisterEncoder(reflect.TypeOf((*func(string) ([]byte, error))(nil)).Elem(), encoderMakerStringE)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, string) []byte)(nil)).Elem(), encoderMakerTString)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, string) ([]byte, error))(nil)).Elem(), encoderMakerTStringE)
+	RegisterEncoder(reflect.TypeOf((*func(int) []byte)(nil)).Elem(), encoderMakerInt)
+	RegisterEncoder(reflect.TypeOf((*func(int) ([]byte, error))(nil)).Elem(), encoderMakerIntE)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, int) []byte)(nil)).Elem(), encoderMakerTInt)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, int) ([]byte, error))(nil)).Elem(), encoderMakerTIntE)
+	RegisterEncoder(reflect.TypeOf((*func(int8) []byte)(nil)).Elem(), encoderMakerInt8)
+	RegisterEncoder(reflect.TypeOf((*func(int8) ([]byte, error))(nil)).Elem(), encoderMakerInt8E)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, int8) []byte)(nil)).Elem(), encoderMakerTInt8)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, int8) ([]byte, error))(nil)).Elem(), encoderMakerTInt8E)
+	RegisterEncoder(reflect.TypeOf((*func(int16) []byte)(nil)).Elem(), encoderMakerInt16)
+	RegisterEncoder(reflect.TypeOf((*func(int16) ([]byte, error))(nil)).Elem(), encoderMakerInt16E)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, int16) []byte)(nil)).Elem(), encoderMakerTInt16)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, int16) ([]byte, error))(nil)).Elem(), encoderMakerTInt16E)
+	RegisterEncoder(reflect.TypeOf((*func(int32) []byte)(nil)).Elem(), encoderMakerInt32)
+	RegisterEncoder(reflect.TypeOf((*func(int32) ([]byte, error))(nil)).Elem(), encoderMakerInt32E)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, int32) []byte)(nil)).Elem(), encoderMakerTInt32)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, int32) ([]byte, error))(nil)).Elem(), encoderMakerTInt32E)
+	RegisterEncoder(reflect.TypeOf((*func(int64) []byte)(nil)).Elem(), encoderMakerInt64)
+	RegisterEncoder(reflect.TypeOf((*func(int64) ([]byte, error))(nil)).Elem(), encoderMakerInt64E)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, int64) []byte)(nil)).Elem(), encoderMakerTInt64)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, int64) ([]byte, error))(nil)).Elem(), encoderMakerTInt64E)
+	RegisterEncoder(reflect.TypeOf((*func(uint) []byte)(nil)).Elem(), encoderMakerUint)
+	RegisterEncoder(reflect.TypeOf((*func(uint) ([]byte, error))(nil)).Elem(), encoderMakerUintE)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, uint) []byte)(nil)).Elem(), encoderMakerTUint)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, uint) ([]byte, error))(nil)).Elem(), encoderMakerTUintE)
+	RegisterEncoder(reflect.TypeOf((*func(uint8) []byte)(nil)).Elem(), encoderMakerUint8)
+	RegisterEncoder(reflect.TypeOf((*func(uint8) ([]byte, error))(nil)).Elem(), encoderMakerUint8E)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, uint8) []byte)(nil)).Elem(), encoderMakerTUint8)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, uint8) ([]byte, error))(nil)).Elem(), encoderMakerTUint8E)
+	RegisterEncoder(reflect.TypeOf((*func(uint16) []byte)(nil)).Elem(), encoderMakerUint16)
+	RegisterEncoder(reflect.TypeOf((*func(uint16) ([]byte, error))(nil)).Elem(), encoderMakerUint16E)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, uint16) []byte)(nil)).Elem(), encoderMakerTUint16)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, uint16) ([]byte, error))(nil)).Elem(), encoderMakerTUint16E)
+	RegisterEncoder(reflect.TypeOf((*func(uint32) []byte)(nil)).Elem(), encoderMakerUint32)
+	RegisterEncoder(reflect.TypeOf((*func(uint32) ([]byte, error))(nil)).Elem(), encoderMakerUint32E)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, uint32) []byte)(nil)).Elem(), encoderMakerTUint32)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, uint32) ([]byte, error))(nil)).Elem(), encoderMakerTUint32E)
+	RegisterEncoder(reflect.TypeOf((*func(uint64) []byte)(nil)).Elem(), encoderMakerUint64)
+	RegisterEncoder(reflect.TypeOf((*func(uint64) ([]byte, error))(nil)).Elem(), encoderMakerUint64E)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, uint64) []byte)(nil)).Elem(), encoderMakerTUint64)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, uint64) ([]byte, error))(nil)).Elem(), encoderMakerTUint64E)
+	RegisterEncoder(reflect.TypeOf((*func(float32) []byte)(nil)).Elem(), encoderMakerFloat32)
+	RegisterEncoder(reflect.TypeOf((*func(float32) ([]byte, error))(nil)).Elem(), encoderMakerFloat32E)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, float32) []byte)(nil)).Elem(), encoderMakerTFloat32)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, float32) ([]byte, error))(nil)).Elem(), encoderMakerTFloat32E)
+	RegisterEncoder(reflect.TypeOf((*func(float64) []byte)(nil)).Elem(), encoderMakerFloat64)
+	RegisterEncoder(reflect.TypeOf((*func(float64) ([]byte, error))(nil)).Elem(), encoderMakerFloat64E)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, float64) []byte)(nil)).Elem(), encoderMakerTFloat64)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, float64) ([]byte, error))(nil)).Elem(), encoderMakerTFloat64E)
+	RegisterEncoder(reflect.TypeOf((*func(typex.T) []byte)(nil)).Elem(), encoderMakerTypex_T)
+	RegisterEncoder(reflect.TypeOf((*func(typex.T) ([]byte, error))(nil)).Elem(), encoderMakerTypex_TE)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, typex.T) []byte)(nil)).Elem(), encoderMakerTTypex_T)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, typex.T) ([]byte, error))(nil)).Elem(), encoderMakerTTypex_TE)
+	RegisterEncoder(reflect.TypeOf((*func(typex.U) []byte)(nil)).Elem(), encoderMakerTypex_U)
+	RegisterEncoder(reflect.TypeOf((*func(typex.U) ([]byte, error))(nil)).Elem(), encoderMakerTypex_UE)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, typex.U) []byte)(nil)).Elem(), encoderMakerTTypex_U)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, typex.U) ([]byte, error))(nil)).Elem(), encoderMakerTTypex_UE)
+	RegisterEncoder(reflect.TypeOf((*func(typex.V) []byte)(nil)).Elem(), encoderMakerTypex_V)
+	RegisterEncoder(reflect.TypeOf((*func(typex.V) ([]byte, error))(nil)).Elem(), encoderMakerTypex_VE)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, typex.V) []byte)(nil)).Elem(), encoderMakerTTypex_V)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, typex.V) ([]byte, error))(nil)).Elem(), encoderMakerTTypex_VE)
+	RegisterEncoder(reflect.TypeOf((*func(typex.W) []byte)(nil)).Elem(), encoderMakerTypex_W)
+	RegisterEncoder(reflect.TypeOf((*func(typex.W) ([]byte, error))(nil)).Elem(), encoderMakerTypex_WE)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, typex.W) []byte)(nil)).Elem(), encoderMakerTTypex_W)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, typex.W) ([]byte, error))(nil)).Elem(), encoderMakerTTypex_WE)
+	RegisterEncoder(reflect.TypeOf((*func(typex.X) []byte)(nil)).Elem(), encoderMakerTypex_X)
+	RegisterEncoder(reflect.TypeOf((*func(typex.X) ([]byte, error))(nil)).Elem(), encoderMakerTypex_XE)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, typex.X) []byte)(nil)).Elem(), encoderMakerTTypex_X)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, typex.X) ([]byte, error))(nil)).Elem(), encoderMakerTTypex_XE)
+	RegisterEncoder(reflect.TypeOf((*func(typex.Y) []byte)(nil)).Elem(), encoderMakerTypex_Y)
+	RegisterEncoder(reflect.TypeOf((*func(typex.Y) ([]byte, error))(nil)).Elem(), encoderMakerTypex_YE)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, typex.Y) []byte)(nil)).Elem(), encoderMakerTTypex_Y)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, typex.Y) ([]byte, error))(nil)).Elem(), encoderMakerTTypex_YE)
+	RegisterEncoder(reflect.TypeOf((*func(typex.Z) []byte)(nil)).Elem(), encoderMakerTypex_Z)
+	RegisterEncoder(reflect.TypeOf((*func(typex.Z) ([]byte, error))(nil)).Elem(), encoderMakerTypex_ZE)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, typex.Z) []byte)(nil)).Elem(), encoderMakerTTypex_Z)
+	RegisterEncoder(reflect.TypeOf((*func(reflect.Type, typex.Z) ([]byte, error))(nil)).Elem(), encoderMakerTTypex_ZE)
+}
+
+type encoderByteSlice struct {
+	fn func([]byte) []byte
+}
+
+func (e *encoderByteSlice) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().([]byte)), nil
+}
+
+func encoderMakerByteSlice(fn reflect.Value) Encoder {
+	return &encoderByteSlice{fn: fn.Interface().(func([]byte) []byte)}
+}
+
+type encoderByteSliceE struct {
+	fn func([]byte) ([]byte, error)
+}
+
+func (e *encoderByteSliceE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().([]byte))
+}
+
+func encoderMakerByteSliceE(fn reflect.Value) Encoder {
+	return &encoderByteSliceE{fn: fn.Interface().(func([]byte) ([]byte, error))}
+}
+
+type encoderTByteSlice struct {
+	fn func(reflect.Type, []byte) []byte
+}
+
+func (e *encoderTByteSlice) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().([]byte)), nil
+}
+
+func encoderMakerTByteSlice(fn reflect.Value) Encoder {
+	return &encoderTByteSlice{fn: fn.Interface().(func(reflect.Type, []byte) []byte)}
+}
+
+type encoderTByteSliceE struct {
+	fn func(reflect.Type, []byte) ([]byte, error)
+}
+
+func (e *encoderTByteSliceE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().([]byte))
+}
+
+func encoderMakerTByteSliceE(fn reflect.Value) Encoder {
+	return &encoderTByteSliceE{fn: fn.Interface().(func(reflect.Type, []byte) ([]byte, error))}
+}
+
+type encoderBool struct {
+	fn func(bool) []byte
+}
+
+func (e *encoderBool) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(bool)), nil
+}
+
+func encoderMakerBool(fn reflect.Value) Encoder {
+	return &encoderBool{fn: fn.Interface().(func(bool) []byte)}
+}
+
+type encoderBoolE struct {
+	fn func(bool) ([]byte, error)
+}
+
+func (e *encoderBoolE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(bool))
+}
+
+func encoderMakerBoolE(fn reflect.Value) Encoder {
+	return &encoderBoolE{fn: fn.Interface().(func(bool) ([]byte, error))}
+}
+
+type encoderTBool struct {
+	fn func(reflect.Type, bool) []byte
+}
+
+func (e *encoderTBool) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(bool)), nil
+}
+
+func encoderMakerTBool(fn reflect.Value) Encoder {
+	return &encoderTBool{fn: fn.Interface().(func(reflect.Type, bool) []byte)}
+}
+
+type encoderTBoolE struct {
+	fn func(reflect.Type, bool) ([]byte, error)
+}
+
+func (e *encoderTBoolE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(bool))
+}
+
+func encoderMakerTBoolE(fn reflect.Value) Encoder {
+	return &encoderTBoolE{fn: fn.Interface().(func(reflect.Type, bool) ([]byte, error))}
+}
+
+type encoderString struct {
+	fn func(string) []byte
+}
+
+func (e *encoderString) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(string)), nil
+}
+
+func encoderMakerString(fn reflect.Value) Encoder {
+	return &encoderString{fn: fn.Interface().(func(string) []byte)}
+}
+
+type encoderStringE struct {
+	fn func(string) ([]byte, error)
+}
+
+func (e *encoderStringE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(string))
+}
+
+func encoderMakerStringE(fn reflect.Value) Encoder {
+	return &encoderStringE{fn: fn.Interface().(func(string) ([]byte, error))}
+}
+
+type encoderTString struct {
+	fn func(reflect.Type, string) []byte
+}
+
+func (e *encoderTString) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(string)), nil
+}
+
+func encoderMakerTString(fn reflect.Value) Encoder {
+	return &encoderTString{fn: fn.Interface().(func(reflect.Type, string) []byte)}
+}
+
+type encoderTStringE struct {
+	fn func(reflect.Type, string) ([]byte, error)
+}
+
+func (e *encoderTStringE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(string))
+}
+
+func encoderMakerTStringE(fn reflect.Value) Encoder {
+	return &encoderTStringE{fn: fn.Interface().(func(reflect.Type, string) ([]byte, error))}
+}
+
+type encoderInt struct {
+	fn func(int) []byte
+}
+
+func (e *encoderInt) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(int)), nil
+}
+
+func encoderMakerInt(fn reflect.Value) Encoder {
+	return &encoderInt{fn: fn.Interface().(func(int) []byte)}
+}
+
+type encoderIntE struct {
+	fn func(int) ([]byte, error)
+}
+
+func (e *encoderIntE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(int))
+}
+
+func encoderMakerIntE(fn reflect.Value) Encoder {
+	return &encoderIntE{fn: fn.Interface().(func(int) ([]byte, error))}
+}
+
+type encoderTInt struct {
+	fn func(reflect.Type, int) []byte
+}
+
+func (e *encoderTInt) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(int)), nil
+}
+
+func encoderMakerTInt(fn reflect.Value) Encoder {
+	return &encoderTInt{fn: fn.Interface().(func(reflect.Type, int) []byte)}
+}
+
+type encoderTIntE struct {
+	fn func(reflect.Type, int) ([]byte, error)
+}
+
+func (e *encoderTIntE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(int))
+}
+
+func encoderMakerTIntE(fn reflect.Value) Encoder {
+	return &encoderTIntE{fn: fn.Interface().(func(reflect.Type, int) ([]byte, error))}
+}
+
+type encoderInt8 struct {
+	fn func(int8) []byte
+}
+
+func (e *encoderInt8) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(int8)), nil
+}
+
+func encoderMakerInt8(fn reflect.Value) Encoder {
+	return &encoderInt8{fn: fn.Interface().(func(int8) []byte)}
+}
+
+type encoderInt8E struct {
+	fn func(int8) ([]byte, error)
+}
+
+func (e *encoderInt8E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(int8))
+}
+
+func encoderMakerInt8E(fn reflect.Value) Encoder {
+	return &encoderInt8E{fn: fn.Interface().(func(int8) ([]byte, error))}
+}
+
+type encoderTInt8 struct {
+	fn func(reflect.Type, int8) []byte
+}
+
+func (e *encoderTInt8) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(int8)), nil
+}
+
+func encoderMakerTInt8(fn reflect.Value) Encoder {
+	return &encoderTInt8{fn: fn.Interface().(func(reflect.Type, int8) []byte)}
+}
+
+type encoderTInt8E struct {
+	fn func(reflect.Type, int8) ([]byte, error)
+}
+
+func (e *encoderTInt8E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(int8))
+}
+
+func encoderMakerTInt8E(fn reflect.Value) Encoder {
+	return &encoderTInt8E{fn: fn.Interface().(func(reflect.Type, int8) ([]byte, error))}
+}
+
+type encoderInt16 struct {
+	fn func(int16) []byte
+}
+
+func (e *encoderInt16) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(int16)), nil
+}
+
+func encoderMakerInt16(fn reflect.Value) Encoder {
+	return &encoderInt16{fn: fn.Interface().(func(int16) []byte)}
+}
+
+type encoderInt16E struct {
+	fn func(int16) ([]byte, error)
+}
+
+func (e *encoderInt16E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(int16))
+}
+
+func encoderMakerInt16E(fn reflect.Value) Encoder {
+	return &encoderInt16E{fn: fn.Interface().(func(int16) ([]byte, error))}
+}
+
+type encoderTInt16 struct {
+	fn func(reflect.Type, int16) []byte
+}
+
+func (e *encoderTInt16) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(int16)), nil
+}
+
+func encoderMakerTInt16(fn reflect.Value) Encoder {
+	return &encoderTInt16{fn: fn.Interface().(func(reflect.Type, int16) []byte)}
+}
+
+type encoderTInt16E struct {
+	fn func(reflect.Type, int16) ([]byte, error)
+}
+
+func (e *encoderTInt16E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(int16))
+}
+
+func encoderMakerTInt16E(fn reflect.Value) Encoder {
+	return &encoderTInt16E{fn: fn.Interface().(func(reflect.Type, int16) ([]byte, error))}
+}
+
+type encoderInt32 struct {
+	fn func(int32) []byte
+}
+
+func (e *encoderInt32) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(int32)), nil
+}
+
+func encoderMakerInt32(fn reflect.Value) Encoder {
+	return &encoderInt32{fn: fn.Interface().(func(int32) []byte)}
+}
+
+type encoderInt32E struct {
+	fn func(int32) ([]byte, error)
+}
+
+func (e *encoderInt32E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(int32))
+}
+
+func encoderMakerInt32E(fn reflect.Value) Encoder {
+	return &encoderInt32E{fn: fn.Interface().(func(int32) ([]byte, error))}
+}
+
+type encoderTInt32 struct {
+	fn func(reflect.Type, int32) []byte
+}
+
+func (e *encoderTInt32) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(int32)), nil
+}
+
+func encoderMakerTInt32(fn reflect.Value) Encoder {
+	return &encoderTInt32{fn: fn.Interface().(func(reflect.Type, int32) []byte)}
+}
+
+type encoderTInt32E struct {
+	fn func(reflect.Type, int32) ([]byte, error)
+}
+
+func (e *encoderTInt32E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(int32))
+}
+
+func encoderMakerTInt32E(fn reflect.Value) Encoder {
+	return &encoderTInt32E{fn: fn.Interface().(func(reflect.Type, int32) ([]byte, error))}
+}
+
+type encoderInt64 struct {
+	fn func(int64) []byte
+}
+
+func (e *encoderInt64) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(int64)), nil
+}
+
+func encoderMakerInt64(fn reflect.Value) Encoder {
+	return &encoderInt64{fn: fn.Interface().(func(int64) []byte)}
+}
+
+type encoderInt64E struct {
+	fn func(int64) ([]byte, error)
+}
+
+func (e *encoderInt64E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(int64))
+}
+
+func encoderMakerInt64E(fn reflect.Value) Encoder {
+	return &encoderInt64E{fn: fn.Interface().(func(int64) ([]byte, error))}
+}
+
+type encoderTInt64 struct {
+	fn func(reflect.Type, int64) []byte
+}
+
+func (e *encoderTInt64) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(int64)), nil
+}
+
+func encoderMakerTInt64(fn reflect.Value) Encoder {
+	return &encoderTInt64{fn: fn.Interface().(func(reflect.Type, int64) []byte)}
+}
+
+type encoderTInt64E struct {
+	fn func(reflect.Type, int64) ([]byte, error)
+}
+
+func (e *encoderTInt64E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(int64))
+}
+
+func encoderMakerTInt64E(fn reflect.Value) Encoder {
+	return &encoderTInt64E{fn: fn.Interface().(func(reflect.Type, int64) ([]byte, error))}
+}
+
+type encoderUint struct {
+	fn func(uint) []byte
+}
+
+func (e *encoderUint) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(uint)), nil
+}
+
+func encoderMakerUint(fn reflect.Value) Encoder {
+	return &encoderUint{fn: fn.Interface().(func(uint) []byte)}
+}
+
+type encoderUintE struct {
+	fn func(uint) ([]byte, error)
+}
+
+func (e *encoderUintE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(uint))
+}
+
+func encoderMakerUintE(fn reflect.Value) Encoder {
+	return &encoderUintE{fn: fn.Interface().(func(uint) ([]byte, error))}
+}
+
+type encoderTUint struct {
+	fn func(reflect.Type, uint) []byte
+}
+
+func (e *encoderTUint) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(uint)), nil
+}
+
+func encoderMakerTUint(fn reflect.Value) Encoder {
+	return &encoderTUint{fn: fn.Interface().(func(reflect.Type, uint) []byte)}
+}
+
+type encoderTUintE struct {
+	fn func(reflect.Type, uint) ([]byte, error)
+}
+
+func (e *encoderTUintE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(uint))
+}
+
+func encoderMakerTUintE(fn reflect.Value) Encoder {
+	return &encoderTUintE{fn: fn.Interface().(func(reflect.Type, uint) ([]byte, error))}
+}
+
+type encoderUint8 struct {
+	fn func(uint8) []byte
+}
+
+func (e *encoderUint8) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(uint8)), nil
+}
+
+func encoderMakerUint8(fn reflect.Value) Encoder {
+	return &encoderUint8{fn: fn.Interface().(func(uint8) []byte)}
+}
+
+type encoderUint8E struct {
+	fn func(uint8) ([]byte, error)
+}
+
+func (e *encoderUint8E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(uint8))
+}
+
+func encoderMakerUint8E(fn reflect.Value) Encoder {
+	return &encoderUint8E{fn: fn.Interface().(func(uint8) ([]byte, error))}
+}
+
+type encoderTUint8 struct {
+	fn func(reflect.Type, uint8) []byte
+}
+
+func (e *encoderTUint8) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(uint8)), nil
+}
+
+func encoderMakerTUint8(fn reflect.Value) Encoder {
+	return &encoderTUint8{fn: fn.Interface().(func(reflect.Type, uint8) []byte)}
+}
+
+type encoderTUint8E struct {
+	fn func(reflect.Type, uint8) ([]byte, error)
+}
+
+func (e *encoderTUint8E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(uint8))
+}
+
+func encoderMakerTUint8E(fn reflect.Value) Encoder {
+	return &encoderTUint8E{fn: fn.Interface().(func(reflect.Type, uint8) ([]byte, error))}
+}
+
+type encoderUint16 struct {
+	fn func(uint16) []byte
+}
+
+func (e *encoderUint16) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(uint16)), nil
+}
+
+func encoderMakerUint16(fn reflect.Value) Encoder {
+	return &encoderUint16{fn: fn.Interface().(func(uint16) []byte)}
+}
+
+type encoderUint16E struct {
+	fn func(uint16) ([]byte, error)
+}
+
+func (e *encoderUint16E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(uint16))
+}
+
+func encoderMakerUint16E(fn reflect.Value) Encoder {
+	return &encoderUint16E{fn: fn.Interface().(func(uint16) ([]byte, error))}
+}
+
+type encoderTUint16 struct {
+	fn func(reflect.Type, uint16) []byte
+}
+
+func (e *encoderTUint16) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(uint16)), nil
+}
+
+func encoderMakerTUint16(fn reflect.Value) Encoder {
+	return &encoderTUint16{fn: fn.Interface().(func(reflect.Type, uint16) []byte)}
+}
+
+type encoderTUint16E struct {
+	fn func(reflect.Type, uint16) ([]byte, error)
+}
+
+func (e *encoderTUint16E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(uint16))
+}
+
+func encoderMakerTUint16E(fn reflect.Value) Encoder {
+	return &encoderTUint16E{fn: fn.Interface().(func(reflect.Type, uint16) ([]byte, error))}
+}
+
+type encoderUint32 struct {
+	fn func(uint32) []byte
+}
+
+func (e *encoderUint32) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(uint32)), nil
+}
+
+func encoderMakerUint32(fn reflect.Value) Encoder {
+	return &encoderUint32{fn: fn.Interface().(func(uint32) []byte)}
+}
+
+type encoderUint32E struct {
+	fn func(uint32) ([]byte, error)
+}
+
+func (e *encoderUint32E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(uint32))
+}
+
+func encoderMakerUint32E(fn reflect.Value) Encoder {
+	return &encoderUint32E{fn: fn.Interface().(func(uint32) ([]byte, error))}
+}
+
+type encoderTUint32 struct {
+	fn func(reflect.Type, uint32) []byte
+}
+
+func (e *encoderTUint32) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(uint32)), nil
+}
+
+func encoderMakerTUint32(fn reflect.Value) Encoder {
+	return &encoderTUint32{fn: fn.Interface().(func(reflect.Type, uint32) []byte)}
+}
+
+type encoderTUint32E struct {
+	fn func(reflect.Type, uint32) ([]byte, error)
+}
+
+func (e *encoderTUint32E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(uint32))
+}
+
+func encoderMakerTUint32E(fn reflect.Value) Encoder {
+	return &encoderTUint32E{fn: fn.Interface().(func(reflect.Type, uint32) ([]byte, error))}
+}
+
+type encoderUint64 struct {
+	fn func(uint64) []byte
+}
+
+func (e *encoderUint64) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(uint64)), nil
+}
+
+func encoderMakerUint64(fn reflect.Value) Encoder {
+	return &encoderUint64{fn: fn.Interface().(func(uint64) []byte)}
+}
+
+type encoderUint64E struct {
+	fn func(uint64) ([]byte, error)
+}
+
+func (e *encoderUint64E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(uint64))
+}
+
+func encoderMakerUint64E(fn reflect.Value) Encoder {
+	return &encoderUint64E{fn: fn.Interface().(func(uint64) ([]byte, error))}
+}
+
+type encoderTUint64 struct {
+	fn func(reflect.Type, uint64) []byte
+}
+
+func (e *encoderTUint64) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(uint64)), nil
+}
+
+func encoderMakerTUint64(fn reflect.Value) Encoder {
+	return &encoderTUint64{fn: fn.Interface().(func(reflect.Type, uint64) []byte)}
+}
+
+type encoderTUint64E struct {
+	fn func(reflect.Type, uint64) ([]byte, error)
+}
+
+func (e *encoderTUint64E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(uint64))
+}
+
+func encoderMakerTUint64E(fn reflect.Value) Encoder {
+	return &encoderTUint64E{fn: fn.Interface().(func(reflect.Type, uint64) ([]byte, error))}
+}
+
+type encoderFloat32 struct {
+	fn func(float32) []byte
+}
+
+func (e *encoderFloat32) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(float32)), nil
+}
+
+func encoderMakerFloat32(fn reflect.Value) Encoder {
+	return &encoderFloat32{fn: fn.Interface().(func(float32) []byte)}
+}
+
+type encoderFloat32E struct {
+	fn func(float32) ([]byte, error)
+}
+
+func (e *encoderFloat32E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(float32))
+}
+
+func encoderMakerFloat32E(fn reflect.Value) Encoder {
+	return &encoderFloat32E{fn: fn.Interface().(func(float32) ([]byte, error))}
+}
+
+type encoderTFloat32 struct {
+	fn func(reflect.Type, float32) []byte
+}
+
+func (e *encoderTFloat32) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(float32)), nil
+}
+
+func encoderMakerTFloat32(fn reflect.Value) Encoder {
+	return &encoderTFloat32{fn: fn.Interface().(func(reflect.Type, float32) []byte)}
+}
+
+type encoderTFloat32E struct {
+	fn func(reflect.Type, float32) ([]byte, error)
+}
+
+func (e *encoderTFloat32E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(float32))
+}
+
+func encoderMakerTFloat32E(fn reflect.Value) Encoder {
+	return &encoderTFloat32E{fn: fn.Interface().(func(reflect.Type, float32) ([]byte, error))}
+}
+
+type encoderFloat64 struct {
+	fn func(float64) []byte
+}
+
+func (e *encoderFloat64) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(float64)), nil
+}
+
+func encoderMakerFloat64(fn reflect.Value) Encoder {
+	return &encoderFloat64{fn: fn.Interface().(func(float64) []byte)}
+}
+
+type encoderFloat64E struct {
+	fn func(float64) ([]byte, error)
+}
+
+func (e *encoderFloat64E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(float64))
+}
+
+func encoderMakerFloat64E(fn reflect.Value) Encoder {
+	return &encoderFloat64E{fn: fn.Interface().(func(float64) ([]byte, error))}
+}
+
+type encoderTFloat64 struct {
+	fn func(reflect.Type, float64) []byte
+}
+
+func (e *encoderTFloat64) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(float64)), nil
+}
+
+func encoderMakerTFloat64(fn reflect.Value) Encoder {
+	return &encoderTFloat64{fn: fn.Interface().(func(reflect.Type, float64) []byte)}
+}
+
+type encoderTFloat64E struct {
+	fn func(reflect.Type, float64) ([]byte, error)
+}
+
+func (e *encoderTFloat64E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(float64))
+}
+
+func encoderMakerTFloat64E(fn reflect.Value) Encoder {
+	return &encoderTFloat64E{fn: fn.Interface().(func(reflect.Type, float64) ([]byte, error))}
+}
+
+type encoderTypex_T struct {
+	fn func(typex.T) []byte
+}
+
+func (e *encoderTypex_T) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(typex.T)), nil
+}
+
+func encoderMakerTypex_T(fn reflect.Value) Encoder {
+	return &encoderTypex_T{fn: fn.Interface().(func(typex.T) []byte)}
+}
+
+type encoderTypex_TE struct {
+	fn func(typex.T) ([]byte, error)
+}
+
+func (e *encoderTypex_TE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(typex.T))
+}
+
+func encoderMakerTypex_TE(fn reflect.Value) Encoder {
+	return &encoderTypex_TE{fn: fn.Interface().(func(typex.T) ([]byte, error))}
+}
+
+type encoderTTypex_T struct {
+	fn func(reflect.Type, typex.T) []byte
+}
+
+func (e *encoderTTypex_T) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(typex.T)), nil
+}
+
+func encoderMakerTTypex_T(fn reflect.Value) Encoder {
+	return &encoderTTypex_T{fn: fn.Interface().(func(reflect.Type, typex.T) []byte)}
+}
+
+type encoderTTypex_TE struct {
+	fn func(reflect.Type, typex.T) ([]byte, error)
+}
+
+func (e *encoderTTypex_TE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(typex.T))
+}
+
+func encoderMakerTTypex_TE(fn reflect.Value) Encoder {
+	return &encoderTTypex_TE{fn: fn.Interface().(func(reflect.Type, typex.T) ([]byte, error))}
+}
+
+type encoderTypex_U struct {
+	fn func(typex.U) []byte
+}
+
+func (e *encoderTypex_U) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(typex.U)), nil
+}
+
+func encoderMakerTypex_U(fn reflect.Value) Encoder {
+	return &encoderTypex_U{fn: fn.Interface().(func(typex.U) []byte)}
+}
+
+type encoderTypex_UE struct {
+	fn func(typex.U) ([]byte, error)
+}
+
+func (e *encoderTypex_UE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(typex.U))
+}
+
+func encoderMakerTypex_UE(fn reflect.Value) Encoder {
+	return &encoderTypex_UE{fn: fn.Interface().(func(typex.U) ([]byte, error))}
+}
+
+type encoderTTypex_U struct {
+	fn func(reflect.Type, typex.U) []byte
+}
+
+func (e *encoderTTypex_U) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(typex.U)), nil
+}
+
+func encoderMakerTTypex_U(fn reflect.Value) Encoder {
+	return &encoderTTypex_U{fn: fn.Interface().(func(reflect.Type, typex.U) []byte)}
+}
+
+type encoderTTypex_UE struct {
+	fn func(reflect.Type, typex.U) ([]byte, error)
+}
+
+func (e *encoderTTypex_UE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(typex.U))
+}
+
+func encoderMakerTTypex_UE(fn reflect.Value) Encoder {
+	return &encoderTTypex_UE{fn: fn.Interface().(func(reflect.Type, typex.U) ([]byte, error))}
+}
+
+type encoderTypex_V struct {
+	fn func(typex.V) []byte
+}
+
+func (e *encoderTypex_V) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(typex.V)), nil
+}
+
+func encoderMakerTypex_V(fn reflect.Value) Encoder {
+	return &encoderTypex_V{fn: fn.Interface().(func(typex.V) []byte)}
+}
+
+type encoderTypex_VE struct {
+	fn func(typex.V) ([]byte, error)
+}
+
+func (e *encoderTypex_VE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(typex.V))
+}
+
+func encoderMakerTypex_VE(fn reflect.Value) Encoder {
+	return &encoderTypex_VE{fn: fn.Interface().(func(typex.V) ([]byte, error))}
+}
+
+type encoderTTypex_V struct {
+	fn func(reflect.Type, typex.V) []byte
+}
+
+func (e *encoderTTypex_V) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(typex.V)), nil
+}
+
+func encoderMakerTTypex_V(fn reflect.Value) Encoder {
+	return &encoderTTypex_V{fn: fn.Interface().(func(reflect.Type, typex.V) []byte)}
+}
+
+type encoderTTypex_VE struct {
+	fn func(reflect.Type, typex.V) ([]byte, error)
+}
+
+func (e *encoderTTypex_VE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(typex.V))
+}
+
+func encoderMakerTTypex_VE(fn reflect.Value) Encoder {
+	return &encoderTTypex_VE{fn: fn.Interface().(func(reflect.Type, typex.V) ([]byte, error))}
+}
+
+type encoderTypex_W struct {
+	fn func(typex.W) []byte
+}
+
+func (e *encoderTypex_W) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(typex.W)), nil
+}
+
+func encoderMakerTypex_W(fn reflect.Value) Encoder {
+	return &encoderTypex_W{fn: fn.Interface().(func(typex.W) []byte)}
+}
+
+type encoderTypex_WE struct {
+	fn func(typex.W) ([]byte, error)
+}
+
+func (e *encoderTypex_WE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(typex.W))
+}
+
+func encoderMakerTypex_WE(fn reflect.Value) Encoder {
+	return &encoderTypex_WE{fn: fn.Interface().(func(typex.W) ([]byte, error))}
+}
+
+type encoderTTypex_W struct {
+	fn func(reflect.Type, typex.W) []byte
+}
+
+func (e *encoderTTypex_W) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(typex.W)), nil
+}
+
+func encoderMakerTTypex_W(fn reflect.Value) Encoder {
+	return &encoderTTypex_W{fn: fn.Interface().(func(reflect.Type, typex.W) []byte)}
+}
+
+type encoderTTypex_WE struct {
+	fn func(reflect.Type, typex.W) ([]byte, error)
+}
+
+func (e *encoderTTypex_WE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(typex.W))
+}
+
+func encoderMakerTTypex_WE(fn reflect.Value) Encoder {
+	return &encoderTTypex_WE{fn: fn.Interface().(func(reflect.Type, typex.W) ([]byte, error))}
+}
+
+type encoderTypex_X struct {
+	fn func(typex.X) []byte
+}
+
+func (e *encoderTypex_X) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(typex.X)), nil
+}
+
+func encoderMakerTypex_X(fn reflect.Value) Encoder {
+	return &encoderTypex_X{fn: fn.Interface().(func(typex.X) []byte)}
+}
+
+type encoderTypex_XE struct {
+	fn func(typex.X) ([]byte, error)
+}
+
+func (e *encoderTypex_XE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(typex.X))
+}
+
+func encoderMakerTypex_XE(fn reflect.Value) Encoder {
+	return &encoderTypex_XE{fn: fn.Interface().(func(typex.X) ([]byte, error))}
+}
+
+type encoderTTypex_X struct {
+	fn func(reflect.Type, typex.X) []byte
+}
+
+func (e *encoderTTypex_X) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(typex.X)), nil
+}
+
+func encoderMakerTTypex_X(fn reflect.Value) Encoder {
+	return &encoderTTypex_X{fn: fn.Interface().(func(reflect.Type, typex.X) []byte)}
+}
+
+type encoderTTypex_XE struct {
+	fn func(reflect.Type, typex.X) ([]byte, error)
+}
+
+func (e *encoderTTypex_XE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(typex.X))
+}
+
+func encoderMakerTTypex_XE(fn reflect.Value) Encoder {
+	return &encoderTTypex_XE{fn: fn.Interface().(func(reflect.Type, typex.X) ([]byte, error))}
+}
+
+type encoderTypex_Y struct {
+	fn func(typex.Y) []byte
+}
+
+func (e *encoderTypex_Y) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(typex.Y)), nil
+}
+
+func encoderMakerTypex_Y(fn reflect.Value) Encoder {
+	return &encoderTypex_Y{fn: fn.Interface().(func(typex.Y) []byte)}
+}
+
+type encoderTypex_YE struct {
+	fn func(typex.Y) ([]byte, error)
+}
+
+func (e *encoderTypex_YE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(typex.Y))
+}
+
+func encoderMakerTypex_YE(fn reflect.Value) Encoder {
+	return &encoderTypex_YE{fn: fn.Interface().(func(typex.Y) ([]byte, error))}
+}
+
+type encoderTTypex_Y struct {
+	fn func(reflect.Type, typex.Y) []byte
+}
+
+func (e *encoderTTypex_Y) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(typex.Y)), nil
+}
+
+func encoderMakerTTypex_Y(fn reflect.Value) Encoder {
+	return &encoderTTypex_Y{fn: fn.Interface().(func(reflect.Type, typex.Y) []byte)}
+}
+
+type encoderTTypex_YE struct {
+	fn func(reflect.Type, typex.Y) ([]byte, error)
+}
+
+func (e *encoderTTypex_YE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(typex.Y))
+}
+
+func encoderMakerTTypex_YE(fn reflect.Value) Encoder {
+	return &encoderTTypex_YE{fn: fn.Interface().(func(reflect.Type, typex.Y) ([]byte, error))}
+}
+
+type encoderTypex_Z struct {
+	fn func(typex.Z) []byte
+}
+
+func (e *encoderTypex_Z) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(typex.Z)), nil
+}
+
+func encoderMakerTypex_Z(fn reflect.Value) Encoder {
+	return &encoderTypex_Z{fn: fn.Interface().(func(typex.Z) []byte)}
+}
+
+type encoderTypex_ZE struct {
+	fn func(typex.Z) ([]byte, error)
+}
+
+func (e *encoderTypex_ZE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(val.Interface().(typex.Z))
+}
+
+func encoderMakerTypex_ZE(fn reflect.Value) Encoder {
+	return &encoderTypex_ZE{fn: fn.Interface().(func(typex.Z) ([]byte, error))}
+}
+
+type encoderTTypex_Z struct {
+	fn func(reflect.Type, typex.Z) []byte
+}
+
+func (e *encoderTTypex_Z) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(typex.Z)), nil
+}
+
+func encoderMakerTTypex_Z(fn reflect.Value) Encoder {
+	return &encoderTTypex_Z{fn: fn.Interface().(func(reflect.Type, typex.Z) []byte)}
+}
+
+type encoderTTypex_ZE struct {
+	fn func(reflect.Type, typex.Z) ([]byte, error)
+}
+
+func (e *encoderTTypex_ZE) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+	return e.fn(t, val.Interface().(typex.Z))
+}
+
+func encoderMakerTTypex_ZE(fn reflect.Value) Encoder {
+	return &encoderTTypex_ZE{fn: fn.Interface().(func(reflect.Type, typex.Z) ([]byte, error))}
+}
diff --git a/sdks/go/pkg/beam/core/runtime/exec/encoders.tmpl b/sdks/go/pkg/beam/core/runtime/exec/encoders.tmpl
new file mode 100644
index 00000000000..2796667d742
--- /dev/null
+++ b/sdks/go/pkg/beam/core/runtime/exec/encoders.tmpl
@@ -0,0 +1,81 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package exec
+
+import (
+    "reflect"
+
+  	"github.com/apache/beam/sdks/go/pkg/beam/core/typex"
+)
+
+func init() {
+{{- range $x := .X}}
+    RegisterEncoder(reflect.TypeOf((*func ({{$x.Type}}) []byte)(nil)).Elem(), encoderMaker{{$x.Name}})
+    RegisterEncoder(reflect.TypeOf((*func ({{$x.Type}}) ([]byte, error))(nil)).Elem(), encoderMaker{{$x.Name}}E)
+    RegisterEncoder(reflect.TypeOf((*func (reflect.Type, {{$x.Type}}) []byte)(nil)).Elem(), encoderMakerT{{$x.Name}})
+    RegisterEncoder(reflect.TypeOf((*func (reflect.Type, {{$x.Type}}) ([]byte, error))(nil)).Elem(), encoderMakerT{{$x.Name}}E)
+{{- end}}
+}
+
+{{range $x := .X}}
+type encoder{{$x.Name}} struct {
+    fn func ({{$x.Type}}) []byte
+}
+
+func (e *encoder{{$x.Name}}) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+    return e.fn(val.Interface().({{$x.Type}})), nil
+}
+
+func encoderMaker{{$x.Name}}(fn reflect.Value) Encoder {
+    return &encoder{{$x.Name}}{fn: fn.Interface().(func ({{$x.Type}}) []byte)}
+}
+
+type encoder{{$x.Name}}E struct {
+    fn func ({{$x.Type}}) ([]byte, error)
+}
+
+func (e *encoder{{$x.Name}}E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+    return e.fn(val.Interface().({{$x.Type}}))
+}
+
+func encoderMaker{{$x.Name}}E(fn reflect.Value) Encoder {
+    return &encoder{{$x.Name}}E{fn: fn.Interface().(func ({{$x.Type}}) ([]byte, error))}
+}
+
+type encoderT{{$x.Name}} struct {
+    fn func (reflect.Type, {{$x.Type}}) []byte
+}
+
+func (e *encoderT{{$x.Name}}) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+    return e.fn(t, val.Interface().({{$x.Type}})), nil
+}
+
+func encoderMakerT{{$x.Name}}(fn reflect.Value) Encoder {
+    return &encoderT{{$x.Name}}{fn: fn.Interface().(func (reflect.Type, {{$x.Type}}) []byte)}
+}
+
+type encoderT{{$x.Name}}E struct {
+    fn func (reflect.Type, {{$x.Type}}) ([]byte, error)
+}
+
+func (e *encoderT{{$x.Name}}E) Encode(t reflect.Type, val reflect.Value) ([]byte, error) {
+    return e.fn(t, val.Interface().({{$x.Type}}))
+}
+
+func encoderMakerT{{$x.Name}}E(fn reflect.Value) Encoder {
+    return &encoderT{{$x.Name}}E{fn: fn.Interface().(func (reflect.Type, {{$x.Type}}) ([]byte, error))}
+}
+{{end}}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> Reduce Go runtime reflective overhead
> -------------------------------------
>
>                 Key: BEAM-3388
>                 URL: https://issues.apache.org/jira/browse/BEAM-3388
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-go
>            Reporter: Henning Rohde
>            Assignee: Henning Rohde
>
> Go reflection is slow and we should avoid it in the Go SDK at runtime, when possible -- especially on the fast paths. It seems unlikely that the language runtime/libraries will improve any time soon: https://github.com/golang/go/issues/7818.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)