Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 81950200D51 for ; Fri, 22 Dec 2017 18:22:55 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 7F7C0160C1A; Fri, 22 Dec 2017 17:22:55 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 534FA160C0A for ; Fri, 22 Dec 2017 18:22:49 +0100 (CET) Received: (qmail 30877 invoked by uid 500); 22 Dec 2017 17:22:48 -0000 Mailing-List: contact commits-help@beam.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@beam.apache.org Delivered-To: mailing list commits@beam.apache.org Received: (qmail 30867 invoked by uid 99); 22 Dec 2017 17:22:48 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 22 Dec 2017 17:22:48 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id 8F432180A33 for ; Fri, 22 Dec 2017 17:22:46 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -100.002 X-Spam-Level: X-Spam-Status: No, score=-100.002 tagged_above=-999 required=6.31 tests=[RCVD_IN_DNSWL_NONE=-0.0001, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id Rr1mONkWZVvR for ; Fri, 22 Dec 2017 17:22:16 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 2CD6A5F470 for ; Fri, 22 Dec 2017 17:22:12 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id CD9A9E0248 for ; Fri, 22 Dec 2017 17:22:10 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 78F21240DE for ; Fri, 22 Dec 2017 17:22:10 +0000 (UTC) Date: Fri, 22 Dec 2017 17:22:10 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: commits@beam.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (BEAM-3388) Reduce Go runtime reflective overhead MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Fri, 22 Dec 2017 17:22:55 -0000 [ https://issues.apache.org/jira/browse/BEAM-3388?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16301690#comment-16301690 ] ASF GitHub Bot commented on BEAM-3388: -------------------------------------- lukecwik closed pull request #4314: [BEAM-3388] Add Go SDK type specialization of emitters URL: https://github.com/apache/beam/pull/4314 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/cmd/specialize/main.go b/sdks/go/cmd/specialize/main.go index 666ae07c491..67863041975 100644 --- a/sdks/go/cmd/specialize/main.go +++ b/sdks/go/cmd/specialize/main.go @@ -78,10 +78,19 @@ type Z struct { Type string } -var macros = map[string][]string{ - "integers": []string{"int", "int8", "int16", "int32", "int64", "uint", "uint8", "uint16", "uint32", "uint64"}, - "floats": []string{"float32", "float64"}, -} +var ( + integers = []string{"int", "int8", "int16", "int32", "int64", "uint", "uint8", "uint16", "uint32", "uint64"} + floats = []string{"float32", "float64"} + primitives = append(append([]string{"bool", "string"}, integers...), floats...) + + macros = map[string][]string{ + "integers": integers, + "floats": floats, + "primitives": primitives, + "data": append([]string{"[]byte"}, primitives...), + "universals": []string{"typex.T", "typex.U", "typex.V", "typex.W", "typex.X", "typex.Y", "typex.Z"}, + } +) func usage() { fmt.Fprintf(os.Stderr, "Usage: %v [options] --input=\n", filepath.Base(os.Args[0])) diff --git a/sdks/go/pkg/beam/core/funcx/output.go b/sdks/go/pkg/beam/core/funcx/output.go index 002392ab74f..116ecf17ef8 100644 --- a/sdks/go/pkg/beam/core/funcx/output.go +++ b/sdks/go/pkg/beam/core/funcx/output.go @@ -27,6 +27,13 @@ func IsEmit(t reflect.Type) bool { return ok } +// IsEmitWithEventTime return true iff the supplied type is an +// emitter and the first argument is the optional EventTime. +func IsEmitWithEventTime(t reflect.Type) bool { + types, ok := UnfoldEmit(t) + return ok && types[0] == typex.EventTimeType +} + // UnfoldEmit returns the parameter types, if an emitter. For example: // // func (int) returns {int} diff --git a/sdks/go/pkg/beam/core/runtime/exec/emit.go b/sdks/go/pkg/beam/core/runtime/exec/emit.go new file mode 100644 index 00000000000..fb259072e5f --- /dev/null +++ b/sdks/go/pkg/beam/core/runtime/exec/emit.go @@ -0,0 +1,121 @@ +// 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 ( + "context" + "fmt" + "reflect" + "sync" + + "github.com/apache/beam/sdks/go/pkg/beam/core/funcx" + "github.com/apache/beam/sdks/go/pkg/beam/core/typex" +) + +//go:generate specialize --input=emitters.tmpl --x=data,universals --y=data,universals + +// ReusableEmitter is a resettable value needed to hold the implicit context and +// emit event time. +type ReusableEmitter interface { + // Init resets the value. Can be called multiple times. + Init(ctx context.Context, t typex.EventTime) error + // Value returns the side input value. Constant value. + Value() reflect.Value +} + +var ( + emitters = make(map[string]func(ElementProcessor) ReusableEmitter) + emittersMu sync.Mutex +) + +// RegisterEmitter registers an emitter for the given type, such as "func(int)". If +// multiple emitters are registered for the same type, the last registration wins. +func RegisterEmitter(t reflect.Type, maker func(ElementProcessor) ReusableEmitter) { + emittersMu.Lock() + defer emittersMu.Unlock() + + emitters[t.String()] = maker +} + +func makeEmit(t reflect.Type, n ElementProcessor) ReusableEmitter { + emittersMu.Lock() + maker, exists := emitters[t.String()] + emittersMu.Unlock() + + if exists { + return maker(n) + } + + // If no specialized implementation is available, we use the (slower) + // reflection-based one. + + types, ok := funcx.UnfoldEmit(t) + if !ok { + panic(fmt.Sprintf("illegal emit type: %v", t)) + } + + ret := &emitValue{n: n, types: types} + ret.fn = reflect.MakeFunc(t, ret.invoke) + return ret +} + +// emitValue is the reflection-based default emitter implementation. +type emitValue struct { + n ElementProcessor + fn reflect.Value + types []reflect.Type + + ctx context.Context + et typex.EventTime +} + +func (e *emitValue) Init(ctx context.Context, et typex.EventTime) error { + e.ctx = ctx + e.et = et + return nil +} + +func (e *emitValue) Value() reflect.Value { + return e.fn +} + +func (e *emitValue) invoke(args []reflect.Value) []reflect.Value { + value := FullValue{Timestamp: e.et} + isKey := true + for i, t := range e.types { + switch { + case t == typex.EventTimeType: + value.Timestamp = args[i].Interface().(typex.EventTime) + case isKey: + value.Elm = args[i] + isKey = false + default: + value.Elm2 = args[i] + } + } + + if err := e.n.ProcessElement(e.ctx, value); err != nil { + // NOTE(herohde) 12/11/2017: emitters do not return an error, so if there + // are problems we rely on the receiving node capturing the error. + // Furthermore, we panic to quickly halt processing -- a corner-case + // is that this panic unwinds _through_ user code and may be caught or + // ignored, in which case we fall back failing bundle when the error is + // returned by FinishBundle. + + panic(err) + } + return nil +} diff --git a/sdks/go/pkg/beam/core/runtime/exec/emitters.go b/sdks/go/pkg/beam/core/runtime/exec/emitters.go new file mode 100644 index 00000000000..59e8f584a1a --- /dev/null +++ b/sdks/go/pkg/beam/core/runtime/exec/emitters.go @@ -0,0 +1,14214 @@ +// 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 ( + "context" + "reflect" + + "github.com/apache/beam/sdks/go/pkg/beam/core/typex" +) + +func init() { + RegisterEmitter(reflect.TypeOf((*func([]byte))(nil)).Elem(), emitMakerByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte))(nil)).Elem(), emitMakerETByteSlice) + RegisterEmitter(reflect.TypeOf((*func([]byte, []byte))(nil)).Elem(), emitMakerByteSliceByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, []byte))(nil)).Elem(), emitMakerETByteSliceByteSlice) + RegisterEmitter(reflect.TypeOf((*func([]byte, bool))(nil)).Elem(), emitMakerByteSliceBool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, bool))(nil)).Elem(), emitMakerETByteSliceBool) + RegisterEmitter(reflect.TypeOf((*func([]byte, string))(nil)).Elem(), emitMakerByteSliceString) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, string))(nil)).Elem(), emitMakerETByteSliceString) + RegisterEmitter(reflect.TypeOf((*func([]byte, int))(nil)).Elem(), emitMakerByteSliceInt) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, int))(nil)).Elem(), emitMakerETByteSliceInt) + RegisterEmitter(reflect.TypeOf((*func([]byte, int8))(nil)).Elem(), emitMakerByteSliceInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, int8))(nil)).Elem(), emitMakerETByteSliceInt8) + RegisterEmitter(reflect.TypeOf((*func([]byte, int16))(nil)).Elem(), emitMakerByteSliceInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, int16))(nil)).Elem(), emitMakerETByteSliceInt16) + RegisterEmitter(reflect.TypeOf((*func([]byte, int32))(nil)).Elem(), emitMakerByteSliceInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, int32))(nil)).Elem(), emitMakerETByteSliceInt32) + RegisterEmitter(reflect.TypeOf((*func([]byte, int64))(nil)).Elem(), emitMakerByteSliceInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, int64))(nil)).Elem(), emitMakerETByteSliceInt64) + RegisterEmitter(reflect.TypeOf((*func([]byte, uint))(nil)).Elem(), emitMakerByteSliceUint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, uint))(nil)).Elem(), emitMakerETByteSliceUint) + RegisterEmitter(reflect.TypeOf((*func([]byte, uint8))(nil)).Elem(), emitMakerByteSliceUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, uint8))(nil)).Elem(), emitMakerETByteSliceUint8) + RegisterEmitter(reflect.TypeOf((*func([]byte, uint16))(nil)).Elem(), emitMakerByteSliceUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, uint16))(nil)).Elem(), emitMakerETByteSliceUint16) + RegisterEmitter(reflect.TypeOf((*func([]byte, uint32))(nil)).Elem(), emitMakerByteSliceUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, uint32))(nil)).Elem(), emitMakerETByteSliceUint32) + RegisterEmitter(reflect.TypeOf((*func([]byte, uint64))(nil)).Elem(), emitMakerByteSliceUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, uint64))(nil)).Elem(), emitMakerETByteSliceUint64) + RegisterEmitter(reflect.TypeOf((*func([]byte, float32))(nil)).Elem(), emitMakerByteSliceFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, float32))(nil)).Elem(), emitMakerETByteSliceFloat32) + RegisterEmitter(reflect.TypeOf((*func([]byte, float64))(nil)).Elem(), emitMakerByteSliceFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, float64))(nil)).Elem(), emitMakerETByteSliceFloat64) + RegisterEmitter(reflect.TypeOf((*func([]byte, typex.T))(nil)).Elem(), emitMakerByteSliceTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, typex.T))(nil)).Elem(), emitMakerETByteSliceTypex_T) + RegisterEmitter(reflect.TypeOf((*func([]byte, typex.U))(nil)).Elem(), emitMakerByteSliceTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, typex.U))(nil)).Elem(), emitMakerETByteSliceTypex_U) + RegisterEmitter(reflect.TypeOf((*func([]byte, typex.V))(nil)).Elem(), emitMakerByteSliceTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, typex.V))(nil)).Elem(), emitMakerETByteSliceTypex_V) + RegisterEmitter(reflect.TypeOf((*func([]byte, typex.W))(nil)).Elem(), emitMakerByteSliceTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, typex.W))(nil)).Elem(), emitMakerETByteSliceTypex_W) + RegisterEmitter(reflect.TypeOf((*func([]byte, typex.X))(nil)).Elem(), emitMakerByteSliceTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, typex.X))(nil)).Elem(), emitMakerETByteSliceTypex_X) + RegisterEmitter(reflect.TypeOf((*func([]byte, typex.Y))(nil)).Elem(), emitMakerByteSliceTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, typex.Y))(nil)).Elem(), emitMakerETByteSliceTypex_Y) + RegisterEmitter(reflect.TypeOf((*func([]byte, typex.Z))(nil)).Elem(), emitMakerByteSliceTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, []byte, typex.Z))(nil)).Elem(), emitMakerETByteSliceTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(bool))(nil)).Elem(), emitMakerBool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool))(nil)).Elem(), emitMakerETBool) + RegisterEmitter(reflect.TypeOf((*func(bool, []byte))(nil)).Elem(), emitMakerBoolByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, []byte))(nil)).Elem(), emitMakerETBoolByteSlice) + RegisterEmitter(reflect.TypeOf((*func(bool, bool))(nil)).Elem(), emitMakerBoolBool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, bool))(nil)).Elem(), emitMakerETBoolBool) + RegisterEmitter(reflect.TypeOf((*func(bool, string))(nil)).Elem(), emitMakerBoolString) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, string))(nil)).Elem(), emitMakerETBoolString) + RegisterEmitter(reflect.TypeOf((*func(bool, int))(nil)).Elem(), emitMakerBoolInt) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, int))(nil)).Elem(), emitMakerETBoolInt) + RegisterEmitter(reflect.TypeOf((*func(bool, int8))(nil)).Elem(), emitMakerBoolInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, int8))(nil)).Elem(), emitMakerETBoolInt8) + RegisterEmitter(reflect.TypeOf((*func(bool, int16))(nil)).Elem(), emitMakerBoolInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, int16))(nil)).Elem(), emitMakerETBoolInt16) + RegisterEmitter(reflect.TypeOf((*func(bool, int32))(nil)).Elem(), emitMakerBoolInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, int32))(nil)).Elem(), emitMakerETBoolInt32) + RegisterEmitter(reflect.TypeOf((*func(bool, int64))(nil)).Elem(), emitMakerBoolInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, int64))(nil)).Elem(), emitMakerETBoolInt64) + RegisterEmitter(reflect.TypeOf((*func(bool, uint))(nil)).Elem(), emitMakerBoolUint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, uint))(nil)).Elem(), emitMakerETBoolUint) + RegisterEmitter(reflect.TypeOf((*func(bool, uint8))(nil)).Elem(), emitMakerBoolUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, uint8))(nil)).Elem(), emitMakerETBoolUint8) + RegisterEmitter(reflect.TypeOf((*func(bool, uint16))(nil)).Elem(), emitMakerBoolUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, uint16))(nil)).Elem(), emitMakerETBoolUint16) + RegisterEmitter(reflect.TypeOf((*func(bool, uint32))(nil)).Elem(), emitMakerBoolUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, uint32))(nil)).Elem(), emitMakerETBoolUint32) + RegisterEmitter(reflect.TypeOf((*func(bool, uint64))(nil)).Elem(), emitMakerBoolUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, uint64))(nil)).Elem(), emitMakerETBoolUint64) + RegisterEmitter(reflect.TypeOf((*func(bool, float32))(nil)).Elem(), emitMakerBoolFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, float32))(nil)).Elem(), emitMakerETBoolFloat32) + RegisterEmitter(reflect.TypeOf((*func(bool, float64))(nil)).Elem(), emitMakerBoolFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, float64))(nil)).Elem(), emitMakerETBoolFloat64) + RegisterEmitter(reflect.TypeOf((*func(bool, typex.T))(nil)).Elem(), emitMakerBoolTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, typex.T))(nil)).Elem(), emitMakerETBoolTypex_T) + RegisterEmitter(reflect.TypeOf((*func(bool, typex.U))(nil)).Elem(), emitMakerBoolTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, typex.U))(nil)).Elem(), emitMakerETBoolTypex_U) + RegisterEmitter(reflect.TypeOf((*func(bool, typex.V))(nil)).Elem(), emitMakerBoolTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, typex.V))(nil)).Elem(), emitMakerETBoolTypex_V) + RegisterEmitter(reflect.TypeOf((*func(bool, typex.W))(nil)).Elem(), emitMakerBoolTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, typex.W))(nil)).Elem(), emitMakerETBoolTypex_W) + RegisterEmitter(reflect.TypeOf((*func(bool, typex.X))(nil)).Elem(), emitMakerBoolTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, typex.X))(nil)).Elem(), emitMakerETBoolTypex_X) + RegisterEmitter(reflect.TypeOf((*func(bool, typex.Y))(nil)).Elem(), emitMakerBoolTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, typex.Y))(nil)).Elem(), emitMakerETBoolTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(bool, typex.Z))(nil)).Elem(), emitMakerBoolTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, bool, typex.Z))(nil)).Elem(), emitMakerETBoolTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(string))(nil)).Elem(), emitMakerString) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string))(nil)).Elem(), emitMakerETString) + RegisterEmitter(reflect.TypeOf((*func(string, []byte))(nil)).Elem(), emitMakerStringByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, []byte))(nil)).Elem(), emitMakerETStringByteSlice) + RegisterEmitter(reflect.TypeOf((*func(string, bool))(nil)).Elem(), emitMakerStringBool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, bool))(nil)).Elem(), emitMakerETStringBool) + RegisterEmitter(reflect.TypeOf((*func(string, string))(nil)).Elem(), emitMakerStringString) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, string))(nil)).Elem(), emitMakerETStringString) + RegisterEmitter(reflect.TypeOf((*func(string, int))(nil)).Elem(), emitMakerStringInt) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, int))(nil)).Elem(), emitMakerETStringInt) + RegisterEmitter(reflect.TypeOf((*func(string, int8))(nil)).Elem(), emitMakerStringInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, int8))(nil)).Elem(), emitMakerETStringInt8) + RegisterEmitter(reflect.TypeOf((*func(string, int16))(nil)).Elem(), emitMakerStringInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, int16))(nil)).Elem(), emitMakerETStringInt16) + RegisterEmitter(reflect.TypeOf((*func(string, int32))(nil)).Elem(), emitMakerStringInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, int32))(nil)).Elem(), emitMakerETStringInt32) + RegisterEmitter(reflect.TypeOf((*func(string, int64))(nil)).Elem(), emitMakerStringInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, int64))(nil)).Elem(), emitMakerETStringInt64) + RegisterEmitter(reflect.TypeOf((*func(string, uint))(nil)).Elem(), emitMakerStringUint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, uint))(nil)).Elem(), emitMakerETStringUint) + RegisterEmitter(reflect.TypeOf((*func(string, uint8))(nil)).Elem(), emitMakerStringUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, uint8))(nil)).Elem(), emitMakerETStringUint8) + RegisterEmitter(reflect.TypeOf((*func(string, uint16))(nil)).Elem(), emitMakerStringUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, uint16))(nil)).Elem(), emitMakerETStringUint16) + RegisterEmitter(reflect.TypeOf((*func(string, uint32))(nil)).Elem(), emitMakerStringUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, uint32))(nil)).Elem(), emitMakerETStringUint32) + RegisterEmitter(reflect.TypeOf((*func(string, uint64))(nil)).Elem(), emitMakerStringUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, uint64))(nil)).Elem(), emitMakerETStringUint64) + RegisterEmitter(reflect.TypeOf((*func(string, float32))(nil)).Elem(), emitMakerStringFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, float32))(nil)).Elem(), emitMakerETStringFloat32) + RegisterEmitter(reflect.TypeOf((*func(string, float64))(nil)).Elem(), emitMakerStringFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, float64))(nil)).Elem(), emitMakerETStringFloat64) + RegisterEmitter(reflect.TypeOf((*func(string, typex.T))(nil)).Elem(), emitMakerStringTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, typex.T))(nil)).Elem(), emitMakerETStringTypex_T) + RegisterEmitter(reflect.TypeOf((*func(string, typex.U))(nil)).Elem(), emitMakerStringTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, typex.U))(nil)).Elem(), emitMakerETStringTypex_U) + RegisterEmitter(reflect.TypeOf((*func(string, typex.V))(nil)).Elem(), emitMakerStringTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, typex.V))(nil)).Elem(), emitMakerETStringTypex_V) + RegisterEmitter(reflect.TypeOf((*func(string, typex.W))(nil)).Elem(), emitMakerStringTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, typex.W))(nil)).Elem(), emitMakerETStringTypex_W) + RegisterEmitter(reflect.TypeOf((*func(string, typex.X))(nil)).Elem(), emitMakerStringTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, typex.X))(nil)).Elem(), emitMakerETStringTypex_X) + RegisterEmitter(reflect.TypeOf((*func(string, typex.Y))(nil)).Elem(), emitMakerStringTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, typex.Y))(nil)).Elem(), emitMakerETStringTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(string, typex.Z))(nil)).Elem(), emitMakerStringTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, string, typex.Z))(nil)).Elem(), emitMakerETStringTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(int))(nil)).Elem(), emitMakerInt) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int))(nil)).Elem(), emitMakerETInt) + RegisterEmitter(reflect.TypeOf((*func(int, []byte))(nil)).Elem(), emitMakerIntByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, []byte))(nil)).Elem(), emitMakerETIntByteSlice) + RegisterEmitter(reflect.TypeOf((*func(int, bool))(nil)).Elem(), emitMakerIntBool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, bool))(nil)).Elem(), emitMakerETIntBool) + RegisterEmitter(reflect.TypeOf((*func(int, string))(nil)).Elem(), emitMakerIntString) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, string))(nil)).Elem(), emitMakerETIntString) + RegisterEmitter(reflect.TypeOf((*func(int, int))(nil)).Elem(), emitMakerIntInt) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, int))(nil)).Elem(), emitMakerETIntInt) + RegisterEmitter(reflect.TypeOf((*func(int, int8))(nil)).Elem(), emitMakerIntInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, int8))(nil)).Elem(), emitMakerETIntInt8) + RegisterEmitter(reflect.TypeOf((*func(int, int16))(nil)).Elem(), emitMakerIntInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, int16))(nil)).Elem(), emitMakerETIntInt16) + RegisterEmitter(reflect.TypeOf((*func(int, int32))(nil)).Elem(), emitMakerIntInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, int32))(nil)).Elem(), emitMakerETIntInt32) + RegisterEmitter(reflect.TypeOf((*func(int, int64))(nil)).Elem(), emitMakerIntInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, int64))(nil)).Elem(), emitMakerETIntInt64) + RegisterEmitter(reflect.TypeOf((*func(int, uint))(nil)).Elem(), emitMakerIntUint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, uint))(nil)).Elem(), emitMakerETIntUint) + RegisterEmitter(reflect.TypeOf((*func(int, uint8))(nil)).Elem(), emitMakerIntUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, uint8))(nil)).Elem(), emitMakerETIntUint8) + RegisterEmitter(reflect.TypeOf((*func(int, uint16))(nil)).Elem(), emitMakerIntUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, uint16))(nil)).Elem(), emitMakerETIntUint16) + RegisterEmitter(reflect.TypeOf((*func(int, uint32))(nil)).Elem(), emitMakerIntUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, uint32))(nil)).Elem(), emitMakerETIntUint32) + RegisterEmitter(reflect.TypeOf((*func(int, uint64))(nil)).Elem(), emitMakerIntUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, uint64))(nil)).Elem(), emitMakerETIntUint64) + RegisterEmitter(reflect.TypeOf((*func(int, float32))(nil)).Elem(), emitMakerIntFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, float32))(nil)).Elem(), emitMakerETIntFloat32) + RegisterEmitter(reflect.TypeOf((*func(int, float64))(nil)).Elem(), emitMakerIntFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, float64))(nil)).Elem(), emitMakerETIntFloat64) + RegisterEmitter(reflect.TypeOf((*func(int, typex.T))(nil)).Elem(), emitMakerIntTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, typex.T))(nil)).Elem(), emitMakerETIntTypex_T) + RegisterEmitter(reflect.TypeOf((*func(int, typex.U))(nil)).Elem(), emitMakerIntTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, typex.U))(nil)).Elem(), emitMakerETIntTypex_U) + RegisterEmitter(reflect.TypeOf((*func(int, typex.V))(nil)).Elem(), emitMakerIntTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, typex.V))(nil)).Elem(), emitMakerETIntTypex_V) + RegisterEmitter(reflect.TypeOf((*func(int, typex.W))(nil)).Elem(), emitMakerIntTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, typex.W))(nil)).Elem(), emitMakerETIntTypex_W) + RegisterEmitter(reflect.TypeOf((*func(int, typex.X))(nil)).Elem(), emitMakerIntTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, typex.X))(nil)).Elem(), emitMakerETIntTypex_X) + RegisterEmitter(reflect.TypeOf((*func(int, typex.Y))(nil)).Elem(), emitMakerIntTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, typex.Y))(nil)).Elem(), emitMakerETIntTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(int, typex.Z))(nil)).Elem(), emitMakerIntTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int, typex.Z))(nil)).Elem(), emitMakerETIntTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(int8))(nil)).Elem(), emitMakerInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8))(nil)).Elem(), emitMakerETInt8) + RegisterEmitter(reflect.TypeOf((*func(int8, []byte))(nil)).Elem(), emitMakerInt8ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, []byte))(nil)).Elem(), emitMakerETInt8ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(int8, bool))(nil)).Elem(), emitMakerInt8Bool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, bool))(nil)).Elem(), emitMakerETInt8Bool) + RegisterEmitter(reflect.TypeOf((*func(int8, string))(nil)).Elem(), emitMakerInt8String) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, string))(nil)).Elem(), emitMakerETInt8String) + RegisterEmitter(reflect.TypeOf((*func(int8, int))(nil)).Elem(), emitMakerInt8Int) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, int))(nil)).Elem(), emitMakerETInt8Int) + RegisterEmitter(reflect.TypeOf((*func(int8, int8))(nil)).Elem(), emitMakerInt8Int8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, int8))(nil)).Elem(), emitMakerETInt8Int8) + RegisterEmitter(reflect.TypeOf((*func(int8, int16))(nil)).Elem(), emitMakerInt8Int16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, int16))(nil)).Elem(), emitMakerETInt8Int16) + RegisterEmitter(reflect.TypeOf((*func(int8, int32))(nil)).Elem(), emitMakerInt8Int32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, int32))(nil)).Elem(), emitMakerETInt8Int32) + RegisterEmitter(reflect.TypeOf((*func(int8, int64))(nil)).Elem(), emitMakerInt8Int64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, int64))(nil)).Elem(), emitMakerETInt8Int64) + RegisterEmitter(reflect.TypeOf((*func(int8, uint))(nil)).Elem(), emitMakerInt8Uint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, uint))(nil)).Elem(), emitMakerETInt8Uint) + RegisterEmitter(reflect.TypeOf((*func(int8, uint8))(nil)).Elem(), emitMakerInt8Uint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, uint8))(nil)).Elem(), emitMakerETInt8Uint8) + RegisterEmitter(reflect.TypeOf((*func(int8, uint16))(nil)).Elem(), emitMakerInt8Uint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, uint16))(nil)).Elem(), emitMakerETInt8Uint16) + RegisterEmitter(reflect.TypeOf((*func(int8, uint32))(nil)).Elem(), emitMakerInt8Uint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, uint32))(nil)).Elem(), emitMakerETInt8Uint32) + RegisterEmitter(reflect.TypeOf((*func(int8, uint64))(nil)).Elem(), emitMakerInt8Uint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, uint64))(nil)).Elem(), emitMakerETInt8Uint64) + RegisterEmitter(reflect.TypeOf((*func(int8, float32))(nil)).Elem(), emitMakerInt8Float32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, float32))(nil)).Elem(), emitMakerETInt8Float32) + RegisterEmitter(reflect.TypeOf((*func(int8, float64))(nil)).Elem(), emitMakerInt8Float64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, float64))(nil)).Elem(), emitMakerETInt8Float64) + RegisterEmitter(reflect.TypeOf((*func(int8, typex.T))(nil)).Elem(), emitMakerInt8Typex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, typex.T))(nil)).Elem(), emitMakerETInt8Typex_T) + RegisterEmitter(reflect.TypeOf((*func(int8, typex.U))(nil)).Elem(), emitMakerInt8Typex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, typex.U))(nil)).Elem(), emitMakerETInt8Typex_U) + RegisterEmitter(reflect.TypeOf((*func(int8, typex.V))(nil)).Elem(), emitMakerInt8Typex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, typex.V))(nil)).Elem(), emitMakerETInt8Typex_V) + RegisterEmitter(reflect.TypeOf((*func(int8, typex.W))(nil)).Elem(), emitMakerInt8Typex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, typex.W))(nil)).Elem(), emitMakerETInt8Typex_W) + RegisterEmitter(reflect.TypeOf((*func(int8, typex.X))(nil)).Elem(), emitMakerInt8Typex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, typex.X))(nil)).Elem(), emitMakerETInt8Typex_X) + RegisterEmitter(reflect.TypeOf((*func(int8, typex.Y))(nil)).Elem(), emitMakerInt8Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, typex.Y))(nil)).Elem(), emitMakerETInt8Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(int8, typex.Z))(nil)).Elem(), emitMakerInt8Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int8, typex.Z))(nil)).Elem(), emitMakerETInt8Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(int16))(nil)).Elem(), emitMakerInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16))(nil)).Elem(), emitMakerETInt16) + RegisterEmitter(reflect.TypeOf((*func(int16, []byte))(nil)).Elem(), emitMakerInt16ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, []byte))(nil)).Elem(), emitMakerETInt16ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(int16, bool))(nil)).Elem(), emitMakerInt16Bool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, bool))(nil)).Elem(), emitMakerETInt16Bool) + RegisterEmitter(reflect.TypeOf((*func(int16, string))(nil)).Elem(), emitMakerInt16String) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, string))(nil)).Elem(), emitMakerETInt16String) + RegisterEmitter(reflect.TypeOf((*func(int16, int))(nil)).Elem(), emitMakerInt16Int) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, int))(nil)).Elem(), emitMakerETInt16Int) + RegisterEmitter(reflect.TypeOf((*func(int16, int8))(nil)).Elem(), emitMakerInt16Int8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, int8))(nil)).Elem(), emitMakerETInt16Int8) + RegisterEmitter(reflect.TypeOf((*func(int16, int16))(nil)).Elem(), emitMakerInt16Int16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, int16))(nil)).Elem(), emitMakerETInt16Int16) + RegisterEmitter(reflect.TypeOf((*func(int16, int32))(nil)).Elem(), emitMakerInt16Int32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, int32))(nil)).Elem(), emitMakerETInt16Int32) + RegisterEmitter(reflect.TypeOf((*func(int16, int64))(nil)).Elem(), emitMakerInt16Int64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, int64))(nil)).Elem(), emitMakerETInt16Int64) + RegisterEmitter(reflect.TypeOf((*func(int16, uint))(nil)).Elem(), emitMakerInt16Uint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, uint))(nil)).Elem(), emitMakerETInt16Uint) + RegisterEmitter(reflect.TypeOf((*func(int16, uint8))(nil)).Elem(), emitMakerInt16Uint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, uint8))(nil)).Elem(), emitMakerETInt16Uint8) + RegisterEmitter(reflect.TypeOf((*func(int16, uint16))(nil)).Elem(), emitMakerInt16Uint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, uint16))(nil)).Elem(), emitMakerETInt16Uint16) + RegisterEmitter(reflect.TypeOf((*func(int16, uint32))(nil)).Elem(), emitMakerInt16Uint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, uint32))(nil)).Elem(), emitMakerETInt16Uint32) + RegisterEmitter(reflect.TypeOf((*func(int16, uint64))(nil)).Elem(), emitMakerInt16Uint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, uint64))(nil)).Elem(), emitMakerETInt16Uint64) + RegisterEmitter(reflect.TypeOf((*func(int16, float32))(nil)).Elem(), emitMakerInt16Float32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, float32))(nil)).Elem(), emitMakerETInt16Float32) + RegisterEmitter(reflect.TypeOf((*func(int16, float64))(nil)).Elem(), emitMakerInt16Float64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, float64))(nil)).Elem(), emitMakerETInt16Float64) + RegisterEmitter(reflect.TypeOf((*func(int16, typex.T))(nil)).Elem(), emitMakerInt16Typex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, typex.T))(nil)).Elem(), emitMakerETInt16Typex_T) + RegisterEmitter(reflect.TypeOf((*func(int16, typex.U))(nil)).Elem(), emitMakerInt16Typex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, typex.U))(nil)).Elem(), emitMakerETInt16Typex_U) + RegisterEmitter(reflect.TypeOf((*func(int16, typex.V))(nil)).Elem(), emitMakerInt16Typex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, typex.V))(nil)).Elem(), emitMakerETInt16Typex_V) + RegisterEmitter(reflect.TypeOf((*func(int16, typex.W))(nil)).Elem(), emitMakerInt16Typex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, typex.W))(nil)).Elem(), emitMakerETInt16Typex_W) + RegisterEmitter(reflect.TypeOf((*func(int16, typex.X))(nil)).Elem(), emitMakerInt16Typex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, typex.X))(nil)).Elem(), emitMakerETInt16Typex_X) + RegisterEmitter(reflect.TypeOf((*func(int16, typex.Y))(nil)).Elem(), emitMakerInt16Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, typex.Y))(nil)).Elem(), emitMakerETInt16Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(int16, typex.Z))(nil)).Elem(), emitMakerInt16Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int16, typex.Z))(nil)).Elem(), emitMakerETInt16Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(int32))(nil)).Elem(), emitMakerInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32))(nil)).Elem(), emitMakerETInt32) + RegisterEmitter(reflect.TypeOf((*func(int32, []byte))(nil)).Elem(), emitMakerInt32ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, []byte))(nil)).Elem(), emitMakerETInt32ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(int32, bool))(nil)).Elem(), emitMakerInt32Bool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, bool))(nil)).Elem(), emitMakerETInt32Bool) + RegisterEmitter(reflect.TypeOf((*func(int32, string))(nil)).Elem(), emitMakerInt32String) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, string))(nil)).Elem(), emitMakerETInt32String) + RegisterEmitter(reflect.TypeOf((*func(int32, int))(nil)).Elem(), emitMakerInt32Int) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, int))(nil)).Elem(), emitMakerETInt32Int) + RegisterEmitter(reflect.TypeOf((*func(int32, int8))(nil)).Elem(), emitMakerInt32Int8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, int8))(nil)).Elem(), emitMakerETInt32Int8) + RegisterEmitter(reflect.TypeOf((*func(int32, int16))(nil)).Elem(), emitMakerInt32Int16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, int16))(nil)).Elem(), emitMakerETInt32Int16) + RegisterEmitter(reflect.TypeOf((*func(int32, int32))(nil)).Elem(), emitMakerInt32Int32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, int32))(nil)).Elem(), emitMakerETInt32Int32) + RegisterEmitter(reflect.TypeOf((*func(int32, int64))(nil)).Elem(), emitMakerInt32Int64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, int64))(nil)).Elem(), emitMakerETInt32Int64) + RegisterEmitter(reflect.TypeOf((*func(int32, uint))(nil)).Elem(), emitMakerInt32Uint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, uint))(nil)).Elem(), emitMakerETInt32Uint) + RegisterEmitter(reflect.TypeOf((*func(int32, uint8))(nil)).Elem(), emitMakerInt32Uint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, uint8))(nil)).Elem(), emitMakerETInt32Uint8) + RegisterEmitter(reflect.TypeOf((*func(int32, uint16))(nil)).Elem(), emitMakerInt32Uint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, uint16))(nil)).Elem(), emitMakerETInt32Uint16) + RegisterEmitter(reflect.TypeOf((*func(int32, uint32))(nil)).Elem(), emitMakerInt32Uint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, uint32))(nil)).Elem(), emitMakerETInt32Uint32) + RegisterEmitter(reflect.TypeOf((*func(int32, uint64))(nil)).Elem(), emitMakerInt32Uint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, uint64))(nil)).Elem(), emitMakerETInt32Uint64) + RegisterEmitter(reflect.TypeOf((*func(int32, float32))(nil)).Elem(), emitMakerInt32Float32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, float32))(nil)).Elem(), emitMakerETInt32Float32) + RegisterEmitter(reflect.TypeOf((*func(int32, float64))(nil)).Elem(), emitMakerInt32Float64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, float64))(nil)).Elem(), emitMakerETInt32Float64) + RegisterEmitter(reflect.TypeOf((*func(int32, typex.T))(nil)).Elem(), emitMakerInt32Typex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, typex.T))(nil)).Elem(), emitMakerETInt32Typex_T) + RegisterEmitter(reflect.TypeOf((*func(int32, typex.U))(nil)).Elem(), emitMakerInt32Typex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, typex.U))(nil)).Elem(), emitMakerETInt32Typex_U) + RegisterEmitter(reflect.TypeOf((*func(int32, typex.V))(nil)).Elem(), emitMakerInt32Typex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, typex.V))(nil)).Elem(), emitMakerETInt32Typex_V) + RegisterEmitter(reflect.TypeOf((*func(int32, typex.W))(nil)).Elem(), emitMakerInt32Typex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, typex.W))(nil)).Elem(), emitMakerETInt32Typex_W) + RegisterEmitter(reflect.TypeOf((*func(int32, typex.X))(nil)).Elem(), emitMakerInt32Typex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, typex.X))(nil)).Elem(), emitMakerETInt32Typex_X) + RegisterEmitter(reflect.TypeOf((*func(int32, typex.Y))(nil)).Elem(), emitMakerInt32Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, typex.Y))(nil)).Elem(), emitMakerETInt32Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(int32, typex.Z))(nil)).Elem(), emitMakerInt32Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int32, typex.Z))(nil)).Elem(), emitMakerETInt32Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(int64))(nil)).Elem(), emitMakerInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64))(nil)).Elem(), emitMakerETInt64) + RegisterEmitter(reflect.TypeOf((*func(int64, []byte))(nil)).Elem(), emitMakerInt64ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, []byte))(nil)).Elem(), emitMakerETInt64ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(int64, bool))(nil)).Elem(), emitMakerInt64Bool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, bool))(nil)).Elem(), emitMakerETInt64Bool) + RegisterEmitter(reflect.TypeOf((*func(int64, string))(nil)).Elem(), emitMakerInt64String) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, string))(nil)).Elem(), emitMakerETInt64String) + RegisterEmitter(reflect.TypeOf((*func(int64, int))(nil)).Elem(), emitMakerInt64Int) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, int))(nil)).Elem(), emitMakerETInt64Int) + RegisterEmitter(reflect.TypeOf((*func(int64, int8))(nil)).Elem(), emitMakerInt64Int8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, int8))(nil)).Elem(), emitMakerETInt64Int8) + RegisterEmitter(reflect.TypeOf((*func(int64, int16))(nil)).Elem(), emitMakerInt64Int16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, int16))(nil)).Elem(), emitMakerETInt64Int16) + RegisterEmitter(reflect.TypeOf((*func(int64, int32))(nil)).Elem(), emitMakerInt64Int32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, int32))(nil)).Elem(), emitMakerETInt64Int32) + RegisterEmitter(reflect.TypeOf((*func(int64, int64))(nil)).Elem(), emitMakerInt64Int64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, int64))(nil)).Elem(), emitMakerETInt64Int64) + RegisterEmitter(reflect.TypeOf((*func(int64, uint))(nil)).Elem(), emitMakerInt64Uint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, uint))(nil)).Elem(), emitMakerETInt64Uint) + RegisterEmitter(reflect.TypeOf((*func(int64, uint8))(nil)).Elem(), emitMakerInt64Uint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, uint8))(nil)).Elem(), emitMakerETInt64Uint8) + RegisterEmitter(reflect.TypeOf((*func(int64, uint16))(nil)).Elem(), emitMakerInt64Uint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, uint16))(nil)).Elem(), emitMakerETInt64Uint16) + RegisterEmitter(reflect.TypeOf((*func(int64, uint32))(nil)).Elem(), emitMakerInt64Uint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, uint32))(nil)).Elem(), emitMakerETInt64Uint32) + RegisterEmitter(reflect.TypeOf((*func(int64, uint64))(nil)).Elem(), emitMakerInt64Uint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, uint64))(nil)).Elem(), emitMakerETInt64Uint64) + RegisterEmitter(reflect.TypeOf((*func(int64, float32))(nil)).Elem(), emitMakerInt64Float32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, float32))(nil)).Elem(), emitMakerETInt64Float32) + RegisterEmitter(reflect.TypeOf((*func(int64, float64))(nil)).Elem(), emitMakerInt64Float64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, float64))(nil)).Elem(), emitMakerETInt64Float64) + RegisterEmitter(reflect.TypeOf((*func(int64, typex.T))(nil)).Elem(), emitMakerInt64Typex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, typex.T))(nil)).Elem(), emitMakerETInt64Typex_T) + RegisterEmitter(reflect.TypeOf((*func(int64, typex.U))(nil)).Elem(), emitMakerInt64Typex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, typex.U))(nil)).Elem(), emitMakerETInt64Typex_U) + RegisterEmitter(reflect.TypeOf((*func(int64, typex.V))(nil)).Elem(), emitMakerInt64Typex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, typex.V))(nil)).Elem(), emitMakerETInt64Typex_V) + RegisterEmitter(reflect.TypeOf((*func(int64, typex.W))(nil)).Elem(), emitMakerInt64Typex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, typex.W))(nil)).Elem(), emitMakerETInt64Typex_W) + RegisterEmitter(reflect.TypeOf((*func(int64, typex.X))(nil)).Elem(), emitMakerInt64Typex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, typex.X))(nil)).Elem(), emitMakerETInt64Typex_X) + RegisterEmitter(reflect.TypeOf((*func(int64, typex.Y))(nil)).Elem(), emitMakerInt64Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, typex.Y))(nil)).Elem(), emitMakerETInt64Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(int64, typex.Z))(nil)).Elem(), emitMakerInt64Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, int64, typex.Z))(nil)).Elem(), emitMakerETInt64Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(uint))(nil)).Elem(), emitMakerUint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint))(nil)).Elem(), emitMakerETUint) + RegisterEmitter(reflect.TypeOf((*func(uint, []byte))(nil)).Elem(), emitMakerUintByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, []byte))(nil)).Elem(), emitMakerETUintByteSlice) + RegisterEmitter(reflect.TypeOf((*func(uint, bool))(nil)).Elem(), emitMakerUintBool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, bool))(nil)).Elem(), emitMakerETUintBool) + RegisterEmitter(reflect.TypeOf((*func(uint, string))(nil)).Elem(), emitMakerUintString) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, string))(nil)).Elem(), emitMakerETUintString) + RegisterEmitter(reflect.TypeOf((*func(uint, int))(nil)).Elem(), emitMakerUintInt) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, int))(nil)).Elem(), emitMakerETUintInt) + RegisterEmitter(reflect.TypeOf((*func(uint, int8))(nil)).Elem(), emitMakerUintInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, int8))(nil)).Elem(), emitMakerETUintInt8) + RegisterEmitter(reflect.TypeOf((*func(uint, int16))(nil)).Elem(), emitMakerUintInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, int16))(nil)).Elem(), emitMakerETUintInt16) + RegisterEmitter(reflect.TypeOf((*func(uint, int32))(nil)).Elem(), emitMakerUintInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, int32))(nil)).Elem(), emitMakerETUintInt32) + RegisterEmitter(reflect.TypeOf((*func(uint, int64))(nil)).Elem(), emitMakerUintInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, int64))(nil)).Elem(), emitMakerETUintInt64) + RegisterEmitter(reflect.TypeOf((*func(uint, uint))(nil)).Elem(), emitMakerUintUint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, uint))(nil)).Elem(), emitMakerETUintUint) + RegisterEmitter(reflect.TypeOf((*func(uint, uint8))(nil)).Elem(), emitMakerUintUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, uint8))(nil)).Elem(), emitMakerETUintUint8) + RegisterEmitter(reflect.TypeOf((*func(uint, uint16))(nil)).Elem(), emitMakerUintUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, uint16))(nil)).Elem(), emitMakerETUintUint16) + RegisterEmitter(reflect.TypeOf((*func(uint, uint32))(nil)).Elem(), emitMakerUintUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, uint32))(nil)).Elem(), emitMakerETUintUint32) + RegisterEmitter(reflect.TypeOf((*func(uint, uint64))(nil)).Elem(), emitMakerUintUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, uint64))(nil)).Elem(), emitMakerETUintUint64) + RegisterEmitter(reflect.TypeOf((*func(uint, float32))(nil)).Elem(), emitMakerUintFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, float32))(nil)).Elem(), emitMakerETUintFloat32) + RegisterEmitter(reflect.TypeOf((*func(uint, float64))(nil)).Elem(), emitMakerUintFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, float64))(nil)).Elem(), emitMakerETUintFloat64) + RegisterEmitter(reflect.TypeOf((*func(uint, typex.T))(nil)).Elem(), emitMakerUintTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, typex.T))(nil)).Elem(), emitMakerETUintTypex_T) + RegisterEmitter(reflect.TypeOf((*func(uint, typex.U))(nil)).Elem(), emitMakerUintTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, typex.U))(nil)).Elem(), emitMakerETUintTypex_U) + RegisterEmitter(reflect.TypeOf((*func(uint, typex.V))(nil)).Elem(), emitMakerUintTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, typex.V))(nil)).Elem(), emitMakerETUintTypex_V) + RegisterEmitter(reflect.TypeOf((*func(uint, typex.W))(nil)).Elem(), emitMakerUintTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, typex.W))(nil)).Elem(), emitMakerETUintTypex_W) + RegisterEmitter(reflect.TypeOf((*func(uint, typex.X))(nil)).Elem(), emitMakerUintTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, typex.X))(nil)).Elem(), emitMakerETUintTypex_X) + RegisterEmitter(reflect.TypeOf((*func(uint, typex.Y))(nil)).Elem(), emitMakerUintTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, typex.Y))(nil)).Elem(), emitMakerETUintTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(uint, typex.Z))(nil)).Elem(), emitMakerUintTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint, typex.Z))(nil)).Elem(), emitMakerETUintTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(uint8))(nil)).Elem(), emitMakerUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8))(nil)).Elem(), emitMakerETUint8) + RegisterEmitter(reflect.TypeOf((*func(uint8, []byte))(nil)).Elem(), emitMakerUint8ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, []byte))(nil)).Elem(), emitMakerETUint8ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(uint8, bool))(nil)).Elem(), emitMakerUint8Bool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, bool))(nil)).Elem(), emitMakerETUint8Bool) + RegisterEmitter(reflect.TypeOf((*func(uint8, string))(nil)).Elem(), emitMakerUint8String) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, string))(nil)).Elem(), emitMakerETUint8String) + RegisterEmitter(reflect.TypeOf((*func(uint8, int))(nil)).Elem(), emitMakerUint8Int) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, int))(nil)).Elem(), emitMakerETUint8Int) + RegisterEmitter(reflect.TypeOf((*func(uint8, int8))(nil)).Elem(), emitMakerUint8Int8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, int8))(nil)).Elem(), emitMakerETUint8Int8) + RegisterEmitter(reflect.TypeOf((*func(uint8, int16))(nil)).Elem(), emitMakerUint8Int16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, int16))(nil)).Elem(), emitMakerETUint8Int16) + RegisterEmitter(reflect.TypeOf((*func(uint8, int32))(nil)).Elem(), emitMakerUint8Int32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, int32))(nil)).Elem(), emitMakerETUint8Int32) + RegisterEmitter(reflect.TypeOf((*func(uint8, int64))(nil)).Elem(), emitMakerUint8Int64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, int64))(nil)).Elem(), emitMakerETUint8Int64) + RegisterEmitter(reflect.TypeOf((*func(uint8, uint))(nil)).Elem(), emitMakerUint8Uint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, uint))(nil)).Elem(), emitMakerETUint8Uint) + RegisterEmitter(reflect.TypeOf((*func(uint8, uint8))(nil)).Elem(), emitMakerUint8Uint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, uint8))(nil)).Elem(), emitMakerETUint8Uint8) + RegisterEmitter(reflect.TypeOf((*func(uint8, uint16))(nil)).Elem(), emitMakerUint8Uint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, uint16))(nil)).Elem(), emitMakerETUint8Uint16) + RegisterEmitter(reflect.TypeOf((*func(uint8, uint32))(nil)).Elem(), emitMakerUint8Uint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, uint32))(nil)).Elem(), emitMakerETUint8Uint32) + RegisterEmitter(reflect.TypeOf((*func(uint8, uint64))(nil)).Elem(), emitMakerUint8Uint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, uint64))(nil)).Elem(), emitMakerETUint8Uint64) + RegisterEmitter(reflect.TypeOf((*func(uint8, float32))(nil)).Elem(), emitMakerUint8Float32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, float32))(nil)).Elem(), emitMakerETUint8Float32) + RegisterEmitter(reflect.TypeOf((*func(uint8, float64))(nil)).Elem(), emitMakerUint8Float64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, float64))(nil)).Elem(), emitMakerETUint8Float64) + RegisterEmitter(reflect.TypeOf((*func(uint8, typex.T))(nil)).Elem(), emitMakerUint8Typex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, typex.T))(nil)).Elem(), emitMakerETUint8Typex_T) + RegisterEmitter(reflect.TypeOf((*func(uint8, typex.U))(nil)).Elem(), emitMakerUint8Typex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, typex.U))(nil)).Elem(), emitMakerETUint8Typex_U) + RegisterEmitter(reflect.TypeOf((*func(uint8, typex.V))(nil)).Elem(), emitMakerUint8Typex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, typex.V))(nil)).Elem(), emitMakerETUint8Typex_V) + RegisterEmitter(reflect.TypeOf((*func(uint8, typex.W))(nil)).Elem(), emitMakerUint8Typex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, typex.W))(nil)).Elem(), emitMakerETUint8Typex_W) + RegisterEmitter(reflect.TypeOf((*func(uint8, typex.X))(nil)).Elem(), emitMakerUint8Typex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, typex.X))(nil)).Elem(), emitMakerETUint8Typex_X) + RegisterEmitter(reflect.TypeOf((*func(uint8, typex.Y))(nil)).Elem(), emitMakerUint8Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, typex.Y))(nil)).Elem(), emitMakerETUint8Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(uint8, typex.Z))(nil)).Elem(), emitMakerUint8Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint8, typex.Z))(nil)).Elem(), emitMakerETUint8Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(uint16))(nil)).Elem(), emitMakerUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16))(nil)).Elem(), emitMakerETUint16) + RegisterEmitter(reflect.TypeOf((*func(uint16, []byte))(nil)).Elem(), emitMakerUint16ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, []byte))(nil)).Elem(), emitMakerETUint16ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(uint16, bool))(nil)).Elem(), emitMakerUint16Bool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, bool))(nil)).Elem(), emitMakerETUint16Bool) + RegisterEmitter(reflect.TypeOf((*func(uint16, string))(nil)).Elem(), emitMakerUint16String) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, string))(nil)).Elem(), emitMakerETUint16String) + RegisterEmitter(reflect.TypeOf((*func(uint16, int))(nil)).Elem(), emitMakerUint16Int) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, int))(nil)).Elem(), emitMakerETUint16Int) + RegisterEmitter(reflect.TypeOf((*func(uint16, int8))(nil)).Elem(), emitMakerUint16Int8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, int8))(nil)).Elem(), emitMakerETUint16Int8) + RegisterEmitter(reflect.TypeOf((*func(uint16, int16))(nil)).Elem(), emitMakerUint16Int16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, int16))(nil)).Elem(), emitMakerETUint16Int16) + RegisterEmitter(reflect.TypeOf((*func(uint16, int32))(nil)).Elem(), emitMakerUint16Int32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, int32))(nil)).Elem(), emitMakerETUint16Int32) + RegisterEmitter(reflect.TypeOf((*func(uint16, int64))(nil)).Elem(), emitMakerUint16Int64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, int64))(nil)).Elem(), emitMakerETUint16Int64) + RegisterEmitter(reflect.TypeOf((*func(uint16, uint))(nil)).Elem(), emitMakerUint16Uint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, uint))(nil)).Elem(), emitMakerETUint16Uint) + RegisterEmitter(reflect.TypeOf((*func(uint16, uint8))(nil)).Elem(), emitMakerUint16Uint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, uint8))(nil)).Elem(), emitMakerETUint16Uint8) + RegisterEmitter(reflect.TypeOf((*func(uint16, uint16))(nil)).Elem(), emitMakerUint16Uint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, uint16))(nil)).Elem(), emitMakerETUint16Uint16) + RegisterEmitter(reflect.TypeOf((*func(uint16, uint32))(nil)).Elem(), emitMakerUint16Uint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, uint32))(nil)).Elem(), emitMakerETUint16Uint32) + RegisterEmitter(reflect.TypeOf((*func(uint16, uint64))(nil)).Elem(), emitMakerUint16Uint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, uint64))(nil)).Elem(), emitMakerETUint16Uint64) + RegisterEmitter(reflect.TypeOf((*func(uint16, float32))(nil)).Elem(), emitMakerUint16Float32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, float32))(nil)).Elem(), emitMakerETUint16Float32) + RegisterEmitter(reflect.TypeOf((*func(uint16, float64))(nil)).Elem(), emitMakerUint16Float64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, float64))(nil)).Elem(), emitMakerETUint16Float64) + RegisterEmitter(reflect.TypeOf((*func(uint16, typex.T))(nil)).Elem(), emitMakerUint16Typex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, typex.T))(nil)).Elem(), emitMakerETUint16Typex_T) + RegisterEmitter(reflect.TypeOf((*func(uint16, typex.U))(nil)).Elem(), emitMakerUint16Typex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, typex.U))(nil)).Elem(), emitMakerETUint16Typex_U) + RegisterEmitter(reflect.TypeOf((*func(uint16, typex.V))(nil)).Elem(), emitMakerUint16Typex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, typex.V))(nil)).Elem(), emitMakerETUint16Typex_V) + RegisterEmitter(reflect.TypeOf((*func(uint16, typex.W))(nil)).Elem(), emitMakerUint16Typex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, typex.W))(nil)).Elem(), emitMakerETUint16Typex_W) + RegisterEmitter(reflect.TypeOf((*func(uint16, typex.X))(nil)).Elem(), emitMakerUint16Typex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, typex.X))(nil)).Elem(), emitMakerETUint16Typex_X) + RegisterEmitter(reflect.TypeOf((*func(uint16, typex.Y))(nil)).Elem(), emitMakerUint16Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, typex.Y))(nil)).Elem(), emitMakerETUint16Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(uint16, typex.Z))(nil)).Elem(), emitMakerUint16Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint16, typex.Z))(nil)).Elem(), emitMakerETUint16Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(uint32))(nil)).Elem(), emitMakerUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32))(nil)).Elem(), emitMakerETUint32) + RegisterEmitter(reflect.TypeOf((*func(uint32, []byte))(nil)).Elem(), emitMakerUint32ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, []byte))(nil)).Elem(), emitMakerETUint32ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(uint32, bool))(nil)).Elem(), emitMakerUint32Bool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, bool))(nil)).Elem(), emitMakerETUint32Bool) + RegisterEmitter(reflect.TypeOf((*func(uint32, string))(nil)).Elem(), emitMakerUint32String) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, string))(nil)).Elem(), emitMakerETUint32String) + RegisterEmitter(reflect.TypeOf((*func(uint32, int))(nil)).Elem(), emitMakerUint32Int) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, int))(nil)).Elem(), emitMakerETUint32Int) + RegisterEmitter(reflect.TypeOf((*func(uint32, int8))(nil)).Elem(), emitMakerUint32Int8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, int8))(nil)).Elem(), emitMakerETUint32Int8) + RegisterEmitter(reflect.TypeOf((*func(uint32, int16))(nil)).Elem(), emitMakerUint32Int16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, int16))(nil)).Elem(), emitMakerETUint32Int16) + RegisterEmitter(reflect.TypeOf((*func(uint32, int32))(nil)).Elem(), emitMakerUint32Int32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, int32))(nil)).Elem(), emitMakerETUint32Int32) + RegisterEmitter(reflect.TypeOf((*func(uint32, int64))(nil)).Elem(), emitMakerUint32Int64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, int64))(nil)).Elem(), emitMakerETUint32Int64) + RegisterEmitter(reflect.TypeOf((*func(uint32, uint))(nil)).Elem(), emitMakerUint32Uint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, uint))(nil)).Elem(), emitMakerETUint32Uint) + RegisterEmitter(reflect.TypeOf((*func(uint32, uint8))(nil)).Elem(), emitMakerUint32Uint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, uint8))(nil)).Elem(), emitMakerETUint32Uint8) + RegisterEmitter(reflect.TypeOf((*func(uint32, uint16))(nil)).Elem(), emitMakerUint32Uint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, uint16))(nil)).Elem(), emitMakerETUint32Uint16) + RegisterEmitter(reflect.TypeOf((*func(uint32, uint32))(nil)).Elem(), emitMakerUint32Uint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, uint32))(nil)).Elem(), emitMakerETUint32Uint32) + RegisterEmitter(reflect.TypeOf((*func(uint32, uint64))(nil)).Elem(), emitMakerUint32Uint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, uint64))(nil)).Elem(), emitMakerETUint32Uint64) + RegisterEmitter(reflect.TypeOf((*func(uint32, float32))(nil)).Elem(), emitMakerUint32Float32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, float32))(nil)).Elem(), emitMakerETUint32Float32) + RegisterEmitter(reflect.TypeOf((*func(uint32, float64))(nil)).Elem(), emitMakerUint32Float64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, float64))(nil)).Elem(), emitMakerETUint32Float64) + RegisterEmitter(reflect.TypeOf((*func(uint32, typex.T))(nil)).Elem(), emitMakerUint32Typex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, typex.T))(nil)).Elem(), emitMakerETUint32Typex_T) + RegisterEmitter(reflect.TypeOf((*func(uint32, typex.U))(nil)).Elem(), emitMakerUint32Typex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, typex.U))(nil)).Elem(), emitMakerETUint32Typex_U) + RegisterEmitter(reflect.TypeOf((*func(uint32, typex.V))(nil)).Elem(), emitMakerUint32Typex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, typex.V))(nil)).Elem(), emitMakerETUint32Typex_V) + RegisterEmitter(reflect.TypeOf((*func(uint32, typex.W))(nil)).Elem(), emitMakerUint32Typex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, typex.W))(nil)).Elem(), emitMakerETUint32Typex_W) + RegisterEmitter(reflect.TypeOf((*func(uint32, typex.X))(nil)).Elem(), emitMakerUint32Typex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, typex.X))(nil)).Elem(), emitMakerETUint32Typex_X) + RegisterEmitter(reflect.TypeOf((*func(uint32, typex.Y))(nil)).Elem(), emitMakerUint32Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, typex.Y))(nil)).Elem(), emitMakerETUint32Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(uint32, typex.Z))(nil)).Elem(), emitMakerUint32Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint32, typex.Z))(nil)).Elem(), emitMakerETUint32Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(uint64))(nil)).Elem(), emitMakerUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64))(nil)).Elem(), emitMakerETUint64) + RegisterEmitter(reflect.TypeOf((*func(uint64, []byte))(nil)).Elem(), emitMakerUint64ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, []byte))(nil)).Elem(), emitMakerETUint64ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(uint64, bool))(nil)).Elem(), emitMakerUint64Bool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, bool))(nil)).Elem(), emitMakerETUint64Bool) + RegisterEmitter(reflect.TypeOf((*func(uint64, string))(nil)).Elem(), emitMakerUint64String) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, string))(nil)).Elem(), emitMakerETUint64String) + RegisterEmitter(reflect.TypeOf((*func(uint64, int))(nil)).Elem(), emitMakerUint64Int) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, int))(nil)).Elem(), emitMakerETUint64Int) + RegisterEmitter(reflect.TypeOf((*func(uint64, int8))(nil)).Elem(), emitMakerUint64Int8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, int8))(nil)).Elem(), emitMakerETUint64Int8) + RegisterEmitter(reflect.TypeOf((*func(uint64, int16))(nil)).Elem(), emitMakerUint64Int16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, int16))(nil)).Elem(), emitMakerETUint64Int16) + RegisterEmitter(reflect.TypeOf((*func(uint64, int32))(nil)).Elem(), emitMakerUint64Int32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, int32))(nil)).Elem(), emitMakerETUint64Int32) + RegisterEmitter(reflect.TypeOf((*func(uint64, int64))(nil)).Elem(), emitMakerUint64Int64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, int64))(nil)).Elem(), emitMakerETUint64Int64) + RegisterEmitter(reflect.TypeOf((*func(uint64, uint))(nil)).Elem(), emitMakerUint64Uint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, uint))(nil)).Elem(), emitMakerETUint64Uint) + RegisterEmitter(reflect.TypeOf((*func(uint64, uint8))(nil)).Elem(), emitMakerUint64Uint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, uint8))(nil)).Elem(), emitMakerETUint64Uint8) + RegisterEmitter(reflect.TypeOf((*func(uint64, uint16))(nil)).Elem(), emitMakerUint64Uint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, uint16))(nil)).Elem(), emitMakerETUint64Uint16) + RegisterEmitter(reflect.TypeOf((*func(uint64, uint32))(nil)).Elem(), emitMakerUint64Uint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, uint32))(nil)).Elem(), emitMakerETUint64Uint32) + RegisterEmitter(reflect.TypeOf((*func(uint64, uint64))(nil)).Elem(), emitMakerUint64Uint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, uint64))(nil)).Elem(), emitMakerETUint64Uint64) + RegisterEmitter(reflect.TypeOf((*func(uint64, float32))(nil)).Elem(), emitMakerUint64Float32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, float32))(nil)).Elem(), emitMakerETUint64Float32) + RegisterEmitter(reflect.TypeOf((*func(uint64, float64))(nil)).Elem(), emitMakerUint64Float64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, float64))(nil)).Elem(), emitMakerETUint64Float64) + RegisterEmitter(reflect.TypeOf((*func(uint64, typex.T))(nil)).Elem(), emitMakerUint64Typex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, typex.T))(nil)).Elem(), emitMakerETUint64Typex_T) + RegisterEmitter(reflect.TypeOf((*func(uint64, typex.U))(nil)).Elem(), emitMakerUint64Typex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, typex.U))(nil)).Elem(), emitMakerETUint64Typex_U) + RegisterEmitter(reflect.TypeOf((*func(uint64, typex.V))(nil)).Elem(), emitMakerUint64Typex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, typex.V))(nil)).Elem(), emitMakerETUint64Typex_V) + RegisterEmitter(reflect.TypeOf((*func(uint64, typex.W))(nil)).Elem(), emitMakerUint64Typex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, typex.W))(nil)).Elem(), emitMakerETUint64Typex_W) + RegisterEmitter(reflect.TypeOf((*func(uint64, typex.X))(nil)).Elem(), emitMakerUint64Typex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, typex.X))(nil)).Elem(), emitMakerETUint64Typex_X) + RegisterEmitter(reflect.TypeOf((*func(uint64, typex.Y))(nil)).Elem(), emitMakerUint64Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, typex.Y))(nil)).Elem(), emitMakerETUint64Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(uint64, typex.Z))(nil)).Elem(), emitMakerUint64Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, uint64, typex.Z))(nil)).Elem(), emitMakerETUint64Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(float32))(nil)).Elem(), emitMakerFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32))(nil)).Elem(), emitMakerETFloat32) + RegisterEmitter(reflect.TypeOf((*func(float32, []byte))(nil)).Elem(), emitMakerFloat32ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, []byte))(nil)).Elem(), emitMakerETFloat32ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(float32, bool))(nil)).Elem(), emitMakerFloat32Bool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, bool))(nil)).Elem(), emitMakerETFloat32Bool) + RegisterEmitter(reflect.TypeOf((*func(float32, string))(nil)).Elem(), emitMakerFloat32String) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, string))(nil)).Elem(), emitMakerETFloat32String) + RegisterEmitter(reflect.TypeOf((*func(float32, int))(nil)).Elem(), emitMakerFloat32Int) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, int))(nil)).Elem(), emitMakerETFloat32Int) + RegisterEmitter(reflect.TypeOf((*func(float32, int8))(nil)).Elem(), emitMakerFloat32Int8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, int8))(nil)).Elem(), emitMakerETFloat32Int8) + RegisterEmitter(reflect.TypeOf((*func(float32, int16))(nil)).Elem(), emitMakerFloat32Int16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, int16))(nil)).Elem(), emitMakerETFloat32Int16) + RegisterEmitter(reflect.TypeOf((*func(float32, int32))(nil)).Elem(), emitMakerFloat32Int32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, int32))(nil)).Elem(), emitMakerETFloat32Int32) + RegisterEmitter(reflect.TypeOf((*func(float32, int64))(nil)).Elem(), emitMakerFloat32Int64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, int64))(nil)).Elem(), emitMakerETFloat32Int64) + RegisterEmitter(reflect.TypeOf((*func(float32, uint))(nil)).Elem(), emitMakerFloat32Uint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, uint))(nil)).Elem(), emitMakerETFloat32Uint) + RegisterEmitter(reflect.TypeOf((*func(float32, uint8))(nil)).Elem(), emitMakerFloat32Uint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, uint8))(nil)).Elem(), emitMakerETFloat32Uint8) + RegisterEmitter(reflect.TypeOf((*func(float32, uint16))(nil)).Elem(), emitMakerFloat32Uint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, uint16))(nil)).Elem(), emitMakerETFloat32Uint16) + RegisterEmitter(reflect.TypeOf((*func(float32, uint32))(nil)).Elem(), emitMakerFloat32Uint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, uint32))(nil)).Elem(), emitMakerETFloat32Uint32) + RegisterEmitter(reflect.TypeOf((*func(float32, uint64))(nil)).Elem(), emitMakerFloat32Uint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, uint64))(nil)).Elem(), emitMakerETFloat32Uint64) + RegisterEmitter(reflect.TypeOf((*func(float32, float32))(nil)).Elem(), emitMakerFloat32Float32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, float32))(nil)).Elem(), emitMakerETFloat32Float32) + RegisterEmitter(reflect.TypeOf((*func(float32, float64))(nil)).Elem(), emitMakerFloat32Float64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, float64))(nil)).Elem(), emitMakerETFloat32Float64) + RegisterEmitter(reflect.TypeOf((*func(float32, typex.T))(nil)).Elem(), emitMakerFloat32Typex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, typex.T))(nil)).Elem(), emitMakerETFloat32Typex_T) + RegisterEmitter(reflect.TypeOf((*func(float32, typex.U))(nil)).Elem(), emitMakerFloat32Typex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, typex.U))(nil)).Elem(), emitMakerETFloat32Typex_U) + RegisterEmitter(reflect.TypeOf((*func(float32, typex.V))(nil)).Elem(), emitMakerFloat32Typex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, typex.V))(nil)).Elem(), emitMakerETFloat32Typex_V) + RegisterEmitter(reflect.TypeOf((*func(float32, typex.W))(nil)).Elem(), emitMakerFloat32Typex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, typex.W))(nil)).Elem(), emitMakerETFloat32Typex_W) + RegisterEmitter(reflect.TypeOf((*func(float32, typex.X))(nil)).Elem(), emitMakerFloat32Typex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, typex.X))(nil)).Elem(), emitMakerETFloat32Typex_X) + RegisterEmitter(reflect.TypeOf((*func(float32, typex.Y))(nil)).Elem(), emitMakerFloat32Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, typex.Y))(nil)).Elem(), emitMakerETFloat32Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(float32, typex.Z))(nil)).Elem(), emitMakerFloat32Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float32, typex.Z))(nil)).Elem(), emitMakerETFloat32Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(float64))(nil)).Elem(), emitMakerFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64))(nil)).Elem(), emitMakerETFloat64) + RegisterEmitter(reflect.TypeOf((*func(float64, []byte))(nil)).Elem(), emitMakerFloat64ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, []byte))(nil)).Elem(), emitMakerETFloat64ByteSlice) + RegisterEmitter(reflect.TypeOf((*func(float64, bool))(nil)).Elem(), emitMakerFloat64Bool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, bool))(nil)).Elem(), emitMakerETFloat64Bool) + RegisterEmitter(reflect.TypeOf((*func(float64, string))(nil)).Elem(), emitMakerFloat64String) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, string))(nil)).Elem(), emitMakerETFloat64String) + RegisterEmitter(reflect.TypeOf((*func(float64, int))(nil)).Elem(), emitMakerFloat64Int) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, int))(nil)).Elem(), emitMakerETFloat64Int) + RegisterEmitter(reflect.TypeOf((*func(float64, int8))(nil)).Elem(), emitMakerFloat64Int8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, int8))(nil)).Elem(), emitMakerETFloat64Int8) + RegisterEmitter(reflect.TypeOf((*func(float64, int16))(nil)).Elem(), emitMakerFloat64Int16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, int16))(nil)).Elem(), emitMakerETFloat64Int16) + RegisterEmitter(reflect.TypeOf((*func(float64, int32))(nil)).Elem(), emitMakerFloat64Int32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, int32))(nil)).Elem(), emitMakerETFloat64Int32) + RegisterEmitter(reflect.TypeOf((*func(float64, int64))(nil)).Elem(), emitMakerFloat64Int64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, int64))(nil)).Elem(), emitMakerETFloat64Int64) + RegisterEmitter(reflect.TypeOf((*func(float64, uint))(nil)).Elem(), emitMakerFloat64Uint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, uint))(nil)).Elem(), emitMakerETFloat64Uint) + RegisterEmitter(reflect.TypeOf((*func(float64, uint8))(nil)).Elem(), emitMakerFloat64Uint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, uint8))(nil)).Elem(), emitMakerETFloat64Uint8) + RegisterEmitter(reflect.TypeOf((*func(float64, uint16))(nil)).Elem(), emitMakerFloat64Uint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, uint16))(nil)).Elem(), emitMakerETFloat64Uint16) + RegisterEmitter(reflect.TypeOf((*func(float64, uint32))(nil)).Elem(), emitMakerFloat64Uint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, uint32))(nil)).Elem(), emitMakerETFloat64Uint32) + RegisterEmitter(reflect.TypeOf((*func(float64, uint64))(nil)).Elem(), emitMakerFloat64Uint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, uint64))(nil)).Elem(), emitMakerETFloat64Uint64) + RegisterEmitter(reflect.TypeOf((*func(float64, float32))(nil)).Elem(), emitMakerFloat64Float32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, float32))(nil)).Elem(), emitMakerETFloat64Float32) + RegisterEmitter(reflect.TypeOf((*func(float64, float64))(nil)).Elem(), emitMakerFloat64Float64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, float64))(nil)).Elem(), emitMakerETFloat64Float64) + RegisterEmitter(reflect.TypeOf((*func(float64, typex.T))(nil)).Elem(), emitMakerFloat64Typex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, typex.T))(nil)).Elem(), emitMakerETFloat64Typex_T) + RegisterEmitter(reflect.TypeOf((*func(float64, typex.U))(nil)).Elem(), emitMakerFloat64Typex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, typex.U))(nil)).Elem(), emitMakerETFloat64Typex_U) + RegisterEmitter(reflect.TypeOf((*func(float64, typex.V))(nil)).Elem(), emitMakerFloat64Typex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, typex.V))(nil)).Elem(), emitMakerETFloat64Typex_V) + RegisterEmitter(reflect.TypeOf((*func(float64, typex.W))(nil)).Elem(), emitMakerFloat64Typex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, typex.W))(nil)).Elem(), emitMakerETFloat64Typex_W) + RegisterEmitter(reflect.TypeOf((*func(float64, typex.X))(nil)).Elem(), emitMakerFloat64Typex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, typex.X))(nil)).Elem(), emitMakerETFloat64Typex_X) + RegisterEmitter(reflect.TypeOf((*func(float64, typex.Y))(nil)).Elem(), emitMakerFloat64Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, typex.Y))(nil)).Elem(), emitMakerETFloat64Typex_Y) + RegisterEmitter(reflect.TypeOf((*func(float64, typex.Z))(nil)).Elem(), emitMakerFloat64Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, float64, typex.Z))(nil)).Elem(), emitMakerETFloat64Typex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.T))(nil)).Elem(), emitMakerTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T))(nil)).Elem(), emitMakerETTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.T, []byte))(nil)).Elem(), emitMakerTypex_TByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, []byte))(nil)).Elem(), emitMakerETTypex_TByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.T, bool))(nil)).Elem(), emitMakerTypex_TBool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, bool))(nil)).Elem(), emitMakerETTypex_TBool) + RegisterEmitter(reflect.TypeOf((*func(typex.T, string))(nil)).Elem(), emitMakerTypex_TString) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, string))(nil)).Elem(), emitMakerETTypex_TString) + RegisterEmitter(reflect.TypeOf((*func(typex.T, int))(nil)).Elem(), emitMakerTypex_TInt) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, int))(nil)).Elem(), emitMakerETTypex_TInt) + RegisterEmitter(reflect.TypeOf((*func(typex.T, int8))(nil)).Elem(), emitMakerTypex_TInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, int8))(nil)).Elem(), emitMakerETTypex_TInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.T, int16))(nil)).Elem(), emitMakerTypex_TInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, int16))(nil)).Elem(), emitMakerETTypex_TInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.T, int32))(nil)).Elem(), emitMakerTypex_TInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, int32))(nil)).Elem(), emitMakerETTypex_TInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.T, int64))(nil)).Elem(), emitMakerTypex_TInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, int64))(nil)).Elem(), emitMakerETTypex_TInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.T, uint))(nil)).Elem(), emitMakerTypex_TUint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, uint))(nil)).Elem(), emitMakerETTypex_TUint) + RegisterEmitter(reflect.TypeOf((*func(typex.T, uint8))(nil)).Elem(), emitMakerTypex_TUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, uint8))(nil)).Elem(), emitMakerETTypex_TUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.T, uint16))(nil)).Elem(), emitMakerTypex_TUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, uint16))(nil)).Elem(), emitMakerETTypex_TUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.T, uint32))(nil)).Elem(), emitMakerTypex_TUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, uint32))(nil)).Elem(), emitMakerETTypex_TUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.T, uint64))(nil)).Elem(), emitMakerTypex_TUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, uint64))(nil)).Elem(), emitMakerETTypex_TUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.T, float32))(nil)).Elem(), emitMakerTypex_TFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, float32))(nil)).Elem(), emitMakerETTypex_TFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.T, float64))(nil)).Elem(), emitMakerTypex_TFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, float64))(nil)).Elem(), emitMakerETTypex_TFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.T, typex.T))(nil)).Elem(), emitMakerTypex_TTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, typex.T))(nil)).Elem(), emitMakerETTypex_TTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.T, typex.U))(nil)).Elem(), emitMakerTypex_TTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, typex.U))(nil)).Elem(), emitMakerETTypex_TTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.T, typex.V))(nil)).Elem(), emitMakerTypex_TTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, typex.V))(nil)).Elem(), emitMakerETTypex_TTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.T, typex.W))(nil)).Elem(), emitMakerTypex_TTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, typex.W))(nil)).Elem(), emitMakerETTypex_TTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.T, typex.X))(nil)).Elem(), emitMakerTypex_TTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, typex.X))(nil)).Elem(), emitMakerETTypex_TTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.T, typex.Y))(nil)).Elem(), emitMakerTypex_TTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, typex.Y))(nil)).Elem(), emitMakerETTypex_TTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.T, typex.Z))(nil)).Elem(), emitMakerTypex_TTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.T, typex.Z))(nil)).Elem(), emitMakerETTypex_TTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.U))(nil)).Elem(), emitMakerTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U))(nil)).Elem(), emitMakerETTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.U, []byte))(nil)).Elem(), emitMakerTypex_UByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, []byte))(nil)).Elem(), emitMakerETTypex_UByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.U, bool))(nil)).Elem(), emitMakerTypex_UBool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, bool))(nil)).Elem(), emitMakerETTypex_UBool) + RegisterEmitter(reflect.TypeOf((*func(typex.U, string))(nil)).Elem(), emitMakerTypex_UString) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, string))(nil)).Elem(), emitMakerETTypex_UString) + RegisterEmitter(reflect.TypeOf((*func(typex.U, int))(nil)).Elem(), emitMakerTypex_UInt) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, int))(nil)).Elem(), emitMakerETTypex_UInt) + RegisterEmitter(reflect.TypeOf((*func(typex.U, int8))(nil)).Elem(), emitMakerTypex_UInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, int8))(nil)).Elem(), emitMakerETTypex_UInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.U, int16))(nil)).Elem(), emitMakerTypex_UInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, int16))(nil)).Elem(), emitMakerETTypex_UInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.U, int32))(nil)).Elem(), emitMakerTypex_UInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, int32))(nil)).Elem(), emitMakerETTypex_UInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.U, int64))(nil)).Elem(), emitMakerTypex_UInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, int64))(nil)).Elem(), emitMakerETTypex_UInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.U, uint))(nil)).Elem(), emitMakerTypex_UUint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, uint))(nil)).Elem(), emitMakerETTypex_UUint) + RegisterEmitter(reflect.TypeOf((*func(typex.U, uint8))(nil)).Elem(), emitMakerTypex_UUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, uint8))(nil)).Elem(), emitMakerETTypex_UUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.U, uint16))(nil)).Elem(), emitMakerTypex_UUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, uint16))(nil)).Elem(), emitMakerETTypex_UUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.U, uint32))(nil)).Elem(), emitMakerTypex_UUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, uint32))(nil)).Elem(), emitMakerETTypex_UUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.U, uint64))(nil)).Elem(), emitMakerTypex_UUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, uint64))(nil)).Elem(), emitMakerETTypex_UUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.U, float32))(nil)).Elem(), emitMakerTypex_UFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, float32))(nil)).Elem(), emitMakerETTypex_UFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.U, float64))(nil)).Elem(), emitMakerTypex_UFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, float64))(nil)).Elem(), emitMakerETTypex_UFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.U, typex.T))(nil)).Elem(), emitMakerTypex_UTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, typex.T))(nil)).Elem(), emitMakerETTypex_UTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.U, typex.U))(nil)).Elem(), emitMakerTypex_UTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, typex.U))(nil)).Elem(), emitMakerETTypex_UTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.U, typex.V))(nil)).Elem(), emitMakerTypex_UTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, typex.V))(nil)).Elem(), emitMakerETTypex_UTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.U, typex.W))(nil)).Elem(), emitMakerTypex_UTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, typex.W))(nil)).Elem(), emitMakerETTypex_UTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.U, typex.X))(nil)).Elem(), emitMakerTypex_UTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, typex.X))(nil)).Elem(), emitMakerETTypex_UTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.U, typex.Y))(nil)).Elem(), emitMakerTypex_UTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, typex.Y))(nil)).Elem(), emitMakerETTypex_UTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.U, typex.Z))(nil)).Elem(), emitMakerTypex_UTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.U, typex.Z))(nil)).Elem(), emitMakerETTypex_UTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.V))(nil)).Elem(), emitMakerTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V))(nil)).Elem(), emitMakerETTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.V, []byte))(nil)).Elem(), emitMakerTypex_VByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, []byte))(nil)).Elem(), emitMakerETTypex_VByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.V, bool))(nil)).Elem(), emitMakerTypex_VBool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, bool))(nil)).Elem(), emitMakerETTypex_VBool) + RegisterEmitter(reflect.TypeOf((*func(typex.V, string))(nil)).Elem(), emitMakerTypex_VString) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, string))(nil)).Elem(), emitMakerETTypex_VString) + RegisterEmitter(reflect.TypeOf((*func(typex.V, int))(nil)).Elem(), emitMakerTypex_VInt) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, int))(nil)).Elem(), emitMakerETTypex_VInt) + RegisterEmitter(reflect.TypeOf((*func(typex.V, int8))(nil)).Elem(), emitMakerTypex_VInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, int8))(nil)).Elem(), emitMakerETTypex_VInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.V, int16))(nil)).Elem(), emitMakerTypex_VInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, int16))(nil)).Elem(), emitMakerETTypex_VInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.V, int32))(nil)).Elem(), emitMakerTypex_VInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, int32))(nil)).Elem(), emitMakerETTypex_VInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.V, int64))(nil)).Elem(), emitMakerTypex_VInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, int64))(nil)).Elem(), emitMakerETTypex_VInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.V, uint))(nil)).Elem(), emitMakerTypex_VUint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, uint))(nil)).Elem(), emitMakerETTypex_VUint) + RegisterEmitter(reflect.TypeOf((*func(typex.V, uint8))(nil)).Elem(), emitMakerTypex_VUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, uint8))(nil)).Elem(), emitMakerETTypex_VUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.V, uint16))(nil)).Elem(), emitMakerTypex_VUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, uint16))(nil)).Elem(), emitMakerETTypex_VUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.V, uint32))(nil)).Elem(), emitMakerTypex_VUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, uint32))(nil)).Elem(), emitMakerETTypex_VUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.V, uint64))(nil)).Elem(), emitMakerTypex_VUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, uint64))(nil)).Elem(), emitMakerETTypex_VUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.V, float32))(nil)).Elem(), emitMakerTypex_VFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, float32))(nil)).Elem(), emitMakerETTypex_VFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.V, float64))(nil)).Elem(), emitMakerTypex_VFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, float64))(nil)).Elem(), emitMakerETTypex_VFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.V, typex.T))(nil)).Elem(), emitMakerTypex_VTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, typex.T))(nil)).Elem(), emitMakerETTypex_VTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.V, typex.U))(nil)).Elem(), emitMakerTypex_VTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, typex.U))(nil)).Elem(), emitMakerETTypex_VTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.V, typex.V))(nil)).Elem(), emitMakerTypex_VTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, typex.V))(nil)).Elem(), emitMakerETTypex_VTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.V, typex.W))(nil)).Elem(), emitMakerTypex_VTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, typex.W))(nil)).Elem(), emitMakerETTypex_VTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.V, typex.X))(nil)).Elem(), emitMakerTypex_VTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, typex.X))(nil)).Elem(), emitMakerETTypex_VTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.V, typex.Y))(nil)).Elem(), emitMakerTypex_VTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, typex.Y))(nil)).Elem(), emitMakerETTypex_VTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.V, typex.Z))(nil)).Elem(), emitMakerTypex_VTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.V, typex.Z))(nil)).Elem(), emitMakerETTypex_VTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.W))(nil)).Elem(), emitMakerTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W))(nil)).Elem(), emitMakerETTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.W, []byte))(nil)).Elem(), emitMakerTypex_WByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, []byte))(nil)).Elem(), emitMakerETTypex_WByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.W, bool))(nil)).Elem(), emitMakerTypex_WBool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, bool))(nil)).Elem(), emitMakerETTypex_WBool) + RegisterEmitter(reflect.TypeOf((*func(typex.W, string))(nil)).Elem(), emitMakerTypex_WString) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, string))(nil)).Elem(), emitMakerETTypex_WString) + RegisterEmitter(reflect.TypeOf((*func(typex.W, int))(nil)).Elem(), emitMakerTypex_WInt) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, int))(nil)).Elem(), emitMakerETTypex_WInt) + RegisterEmitter(reflect.TypeOf((*func(typex.W, int8))(nil)).Elem(), emitMakerTypex_WInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, int8))(nil)).Elem(), emitMakerETTypex_WInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.W, int16))(nil)).Elem(), emitMakerTypex_WInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, int16))(nil)).Elem(), emitMakerETTypex_WInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.W, int32))(nil)).Elem(), emitMakerTypex_WInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, int32))(nil)).Elem(), emitMakerETTypex_WInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.W, int64))(nil)).Elem(), emitMakerTypex_WInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, int64))(nil)).Elem(), emitMakerETTypex_WInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.W, uint))(nil)).Elem(), emitMakerTypex_WUint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, uint))(nil)).Elem(), emitMakerETTypex_WUint) + RegisterEmitter(reflect.TypeOf((*func(typex.W, uint8))(nil)).Elem(), emitMakerTypex_WUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, uint8))(nil)).Elem(), emitMakerETTypex_WUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.W, uint16))(nil)).Elem(), emitMakerTypex_WUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, uint16))(nil)).Elem(), emitMakerETTypex_WUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.W, uint32))(nil)).Elem(), emitMakerTypex_WUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, uint32))(nil)).Elem(), emitMakerETTypex_WUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.W, uint64))(nil)).Elem(), emitMakerTypex_WUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, uint64))(nil)).Elem(), emitMakerETTypex_WUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.W, float32))(nil)).Elem(), emitMakerTypex_WFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, float32))(nil)).Elem(), emitMakerETTypex_WFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.W, float64))(nil)).Elem(), emitMakerTypex_WFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, float64))(nil)).Elem(), emitMakerETTypex_WFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.W, typex.T))(nil)).Elem(), emitMakerTypex_WTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, typex.T))(nil)).Elem(), emitMakerETTypex_WTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.W, typex.U))(nil)).Elem(), emitMakerTypex_WTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, typex.U))(nil)).Elem(), emitMakerETTypex_WTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.W, typex.V))(nil)).Elem(), emitMakerTypex_WTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, typex.V))(nil)).Elem(), emitMakerETTypex_WTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.W, typex.W))(nil)).Elem(), emitMakerTypex_WTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, typex.W))(nil)).Elem(), emitMakerETTypex_WTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.W, typex.X))(nil)).Elem(), emitMakerTypex_WTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, typex.X))(nil)).Elem(), emitMakerETTypex_WTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.W, typex.Y))(nil)).Elem(), emitMakerTypex_WTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, typex.Y))(nil)).Elem(), emitMakerETTypex_WTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.W, typex.Z))(nil)).Elem(), emitMakerTypex_WTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.W, typex.Z))(nil)).Elem(), emitMakerETTypex_WTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.X))(nil)).Elem(), emitMakerTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X))(nil)).Elem(), emitMakerETTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.X, []byte))(nil)).Elem(), emitMakerTypex_XByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, []byte))(nil)).Elem(), emitMakerETTypex_XByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.X, bool))(nil)).Elem(), emitMakerTypex_XBool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, bool))(nil)).Elem(), emitMakerETTypex_XBool) + RegisterEmitter(reflect.TypeOf((*func(typex.X, string))(nil)).Elem(), emitMakerTypex_XString) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, string))(nil)).Elem(), emitMakerETTypex_XString) + RegisterEmitter(reflect.TypeOf((*func(typex.X, int))(nil)).Elem(), emitMakerTypex_XInt) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, int))(nil)).Elem(), emitMakerETTypex_XInt) + RegisterEmitter(reflect.TypeOf((*func(typex.X, int8))(nil)).Elem(), emitMakerTypex_XInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, int8))(nil)).Elem(), emitMakerETTypex_XInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.X, int16))(nil)).Elem(), emitMakerTypex_XInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, int16))(nil)).Elem(), emitMakerETTypex_XInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.X, int32))(nil)).Elem(), emitMakerTypex_XInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, int32))(nil)).Elem(), emitMakerETTypex_XInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.X, int64))(nil)).Elem(), emitMakerTypex_XInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, int64))(nil)).Elem(), emitMakerETTypex_XInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.X, uint))(nil)).Elem(), emitMakerTypex_XUint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, uint))(nil)).Elem(), emitMakerETTypex_XUint) + RegisterEmitter(reflect.TypeOf((*func(typex.X, uint8))(nil)).Elem(), emitMakerTypex_XUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, uint8))(nil)).Elem(), emitMakerETTypex_XUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.X, uint16))(nil)).Elem(), emitMakerTypex_XUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, uint16))(nil)).Elem(), emitMakerETTypex_XUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.X, uint32))(nil)).Elem(), emitMakerTypex_XUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, uint32))(nil)).Elem(), emitMakerETTypex_XUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.X, uint64))(nil)).Elem(), emitMakerTypex_XUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, uint64))(nil)).Elem(), emitMakerETTypex_XUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.X, float32))(nil)).Elem(), emitMakerTypex_XFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, float32))(nil)).Elem(), emitMakerETTypex_XFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.X, float64))(nil)).Elem(), emitMakerTypex_XFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, float64))(nil)).Elem(), emitMakerETTypex_XFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.X, typex.T))(nil)).Elem(), emitMakerTypex_XTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, typex.T))(nil)).Elem(), emitMakerETTypex_XTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.X, typex.U))(nil)).Elem(), emitMakerTypex_XTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, typex.U))(nil)).Elem(), emitMakerETTypex_XTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.X, typex.V))(nil)).Elem(), emitMakerTypex_XTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, typex.V))(nil)).Elem(), emitMakerETTypex_XTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.X, typex.W))(nil)).Elem(), emitMakerTypex_XTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, typex.W))(nil)).Elem(), emitMakerETTypex_XTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.X, typex.X))(nil)).Elem(), emitMakerTypex_XTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, typex.X))(nil)).Elem(), emitMakerETTypex_XTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.X, typex.Y))(nil)).Elem(), emitMakerTypex_XTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, typex.Y))(nil)).Elem(), emitMakerETTypex_XTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.X, typex.Z))(nil)).Elem(), emitMakerTypex_XTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.X, typex.Z))(nil)).Elem(), emitMakerETTypex_XTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.Y))(nil)).Elem(), emitMakerTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y))(nil)).Elem(), emitMakerETTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, []byte))(nil)).Elem(), emitMakerTypex_YByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, []byte))(nil)).Elem(), emitMakerETTypex_YByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, bool))(nil)).Elem(), emitMakerTypex_YBool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, bool))(nil)).Elem(), emitMakerETTypex_YBool) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, string))(nil)).Elem(), emitMakerTypex_YString) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, string))(nil)).Elem(), emitMakerETTypex_YString) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, int))(nil)).Elem(), emitMakerTypex_YInt) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, int))(nil)).Elem(), emitMakerETTypex_YInt) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, int8))(nil)).Elem(), emitMakerTypex_YInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, int8))(nil)).Elem(), emitMakerETTypex_YInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, int16))(nil)).Elem(), emitMakerTypex_YInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, int16))(nil)).Elem(), emitMakerETTypex_YInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, int32))(nil)).Elem(), emitMakerTypex_YInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, int32))(nil)).Elem(), emitMakerETTypex_YInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, int64))(nil)).Elem(), emitMakerTypex_YInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, int64))(nil)).Elem(), emitMakerETTypex_YInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, uint))(nil)).Elem(), emitMakerTypex_YUint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, uint))(nil)).Elem(), emitMakerETTypex_YUint) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, uint8))(nil)).Elem(), emitMakerTypex_YUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, uint8))(nil)).Elem(), emitMakerETTypex_YUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, uint16))(nil)).Elem(), emitMakerTypex_YUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, uint16))(nil)).Elem(), emitMakerETTypex_YUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, uint32))(nil)).Elem(), emitMakerTypex_YUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, uint32))(nil)).Elem(), emitMakerETTypex_YUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, uint64))(nil)).Elem(), emitMakerTypex_YUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, uint64))(nil)).Elem(), emitMakerETTypex_YUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, float32))(nil)).Elem(), emitMakerTypex_YFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, float32))(nil)).Elem(), emitMakerETTypex_YFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, float64))(nil)).Elem(), emitMakerTypex_YFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, float64))(nil)).Elem(), emitMakerETTypex_YFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, typex.T))(nil)).Elem(), emitMakerTypex_YTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, typex.T))(nil)).Elem(), emitMakerETTypex_YTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, typex.U))(nil)).Elem(), emitMakerTypex_YTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, typex.U))(nil)).Elem(), emitMakerETTypex_YTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, typex.V))(nil)).Elem(), emitMakerTypex_YTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, typex.V))(nil)).Elem(), emitMakerETTypex_YTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, typex.W))(nil)).Elem(), emitMakerTypex_YTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, typex.W))(nil)).Elem(), emitMakerETTypex_YTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, typex.X))(nil)).Elem(), emitMakerTypex_YTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, typex.X))(nil)).Elem(), emitMakerETTypex_YTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, typex.Y))(nil)).Elem(), emitMakerTypex_YTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, typex.Y))(nil)).Elem(), emitMakerETTypex_YTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.Y, typex.Z))(nil)).Elem(), emitMakerTypex_YTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Y, typex.Z))(nil)).Elem(), emitMakerETTypex_YTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.Z))(nil)).Elem(), emitMakerTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z))(nil)).Elem(), emitMakerETTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, []byte))(nil)).Elem(), emitMakerTypex_ZByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, []byte))(nil)).Elem(), emitMakerETTypex_ZByteSlice) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, bool))(nil)).Elem(), emitMakerTypex_ZBool) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, bool))(nil)).Elem(), emitMakerETTypex_ZBool) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, string))(nil)).Elem(), emitMakerTypex_ZString) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, string))(nil)).Elem(), emitMakerETTypex_ZString) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, int))(nil)).Elem(), emitMakerTypex_ZInt) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, int))(nil)).Elem(), emitMakerETTypex_ZInt) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, int8))(nil)).Elem(), emitMakerTypex_ZInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, int8))(nil)).Elem(), emitMakerETTypex_ZInt8) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, int16))(nil)).Elem(), emitMakerTypex_ZInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, int16))(nil)).Elem(), emitMakerETTypex_ZInt16) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, int32))(nil)).Elem(), emitMakerTypex_ZInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, int32))(nil)).Elem(), emitMakerETTypex_ZInt32) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, int64))(nil)).Elem(), emitMakerTypex_ZInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, int64))(nil)).Elem(), emitMakerETTypex_ZInt64) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, uint))(nil)).Elem(), emitMakerTypex_ZUint) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, uint))(nil)).Elem(), emitMakerETTypex_ZUint) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, uint8))(nil)).Elem(), emitMakerTypex_ZUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, uint8))(nil)).Elem(), emitMakerETTypex_ZUint8) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, uint16))(nil)).Elem(), emitMakerTypex_ZUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, uint16))(nil)).Elem(), emitMakerETTypex_ZUint16) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, uint32))(nil)).Elem(), emitMakerTypex_ZUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, uint32))(nil)).Elem(), emitMakerETTypex_ZUint32) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, uint64))(nil)).Elem(), emitMakerTypex_ZUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, uint64))(nil)).Elem(), emitMakerETTypex_ZUint64) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, float32))(nil)).Elem(), emitMakerTypex_ZFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, float32))(nil)).Elem(), emitMakerETTypex_ZFloat32) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, float64))(nil)).Elem(), emitMakerTypex_ZFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, float64))(nil)).Elem(), emitMakerETTypex_ZFloat64) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, typex.T))(nil)).Elem(), emitMakerTypex_ZTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, typex.T))(nil)).Elem(), emitMakerETTypex_ZTypex_T) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, typex.U))(nil)).Elem(), emitMakerTypex_ZTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, typex.U))(nil)).Elem(), emitMakerETTypex_ZTypex_U) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, typex.V))(nil)).Elem(), emitMakerTypex_ZTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, typex.V))(nil)).Elem(), emitMakerETTypex_ZTypex_V) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, typex.W))(nil)).Elem(), emitMakerTypex_ZTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, typex.W))(nil)).Elem(), emitMakerETTypex_ZTypex_W) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, typex.X))(nil)).Elem(), emitMakerTypex_ZTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, typex.X))(nil)).Elem(), emitMakerETTypex_ZTypex_X) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, typex.Y))(nil)).Elem(), emitMakerTypex_ZTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, typex.Y))(nil)).Elem(), emitMakerETTypex_ZTypex_Y) + RegisterEmitter(reflect.TypeOf((*func(typex.Z, typex.Z))(nil)).Elem(), emitMakerTypex_ZTypex_Z) + RegisterEmitter(reflect.TypeOf((*func(typex.EventTime, typex.Z, typex.Z))(nil)).Elem(), emitMakerETTypex_ZTypex_Z) +} + +type emitNative struct { + n ElementProcessor + fn reflect.Value + + ctx context.Context + et typex.EventTime +} + +func (e *emitNative) Init(ctx context.Context, et typex.EventTime) error { + e.ctx = ctx + e.et = et + return nil +} + +func (e *emitNative) Value() reflect.Value { + return e.fn +} + +func emitMakerByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSlice) + return ret +} + +func (e *emitNative) invokeByteSlice(elm []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSlice) + return ret +} + +func (e *emitNative) invokeETByteSlice(t typex.EventTime, elm []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceByteSlice) + return ret +} + +func (e *emitNative) invokeByteSliceByteSlice(key []byte, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceByteSlice) + return ret +} + +func (e *emitNative) invokeETByteSliceByteSlice(t typex.EventTime, key []byte, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceBool) + return ret +} + +func (e *emitNative) invokeByteSliceBool(key []byte, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceBool) + return ret +} + +func (e *emitNative) invokeETByteSliceBool(t typex.EventTime, key []byte, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceString) + return ret +} + +func (e *emitNative) invokeByteSliceString(key []byte, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceString) + return ret +} + +func (e *emitNative) invokeETByteSliceString(t typex.EventTime, key []byte, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceInt) + return ret +} + +func (e *emitNative) invokeByteSliceInt(key []byte, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceInt) + return ret +} + +func (e *emitNative) invokeETByteSliceInt(t typex.EventTime, key []byte, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceInt8) + return ret +} + +func (e *emitNative) invokeByteSliceInt8(key []byte, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceInt8) + return ret +} + +func (e *emitNative) invokeETByteSliceInt8(t typex.EventTime, key []byte, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceInt16) + return ret +} + +func (e *emitNative) invokeByteSliceInt16(key []byte, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceInt16) + return ret +} + +func (e *emitNative) invokeETByteSliceInt16(t typex.EventTime, key []byte, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceInt32) + return ret +} + +func (e *emitNative) invokeByteSliceInt32(key []byte, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceInt32) + return ret +} + +func (e *emitNative) invokeETByteSliceInt32(t typex.EventTime, key []byte, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceInt64) + return ret +} + +func (e *emitNative) invokeByteSliceInt64(key []byte, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceInt64) + return ret +} + +func (e *emitNative) invokeETByteSliceInt64(t typex.EventTime, key []byte, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceUint) + return ret +} + +func (e *emitNative) invokeByteSliceUint(key []byte, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceUint) + return ret +} + +func (e *emitNative) invokeETByteSliceUint(t typex.EventTime, key []byte, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceUint8) + return ret +} + +func (e *emitNative) invokeByteSliceUint8(key []byte, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceUint8) + return ret +} + +func (e *emitNative) invokeETByteSliceUint8(t typex.EventTime, key []byte, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceUint16) + return ret +} + +func (e *emitNative) invokeByteSliceUint16(key []byte, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceUint16) + return ret +} + +func (e *emitNative) invokeETByteSliceUint16(t typex.EventTime, key []byte, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceUint32) + return ret +} + +func (e *emitNative) invokeByteSliceUint32(key []byte, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceUint32) + return ret +} + +func (e *emitNative) invokeETByteSliceUint32(t typex.EventTime, key []byte, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceUint64) + return ret +} + +func (e *emitNative) invokeByteSliceUint64(key []byte, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceUint64) + return ret +} + +func (e *emitNative) invokeETByteSliceUint64(t typex.EventTime, key []byte, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceFloat32) + return ret +} + +func (e *emitNative) invokeByteSliceFloat32(key []byte, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceFloat32) + return ret +} + +func (e *emitNative) invokeETByteSliceFloat32(t typex.EventTime, key []byte, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceFloat64) + return ret +} + +func (e *emitNative) invokeByteSliceFloat64(key []byte, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceFloat64) + return ret +} + +func (e *emitNative) invokeETByteSliceFloat64(t typex.EventTime, key []byte, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceTypex_T) + return ret +} + +func (e *emitNative) invokeByteSliceTypex_T(key []byte, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceTypex_T) + return ret +} + +func (e *emitNative) invokeETByteSliceTypex_T(t typex.EventTime, key []byte, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceTypex_U) + return ret +} + +func (e *emitNative) invokeByteSliceTypex_U(key []byte, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceTypex_U) + return ret +} + +func (e *emitNative) invokeETByteSliceTypex_U(t typex.EventTime, key []byte, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceTypex_V) + return ret +} + +func (e *emitNative) invokeByteSliceTypex_V(key []byte, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceTypex_V) + return ret +} + +func (e *emitNative) invokeETByteSliceTypex_V(t typex.EventTime, key []byte, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceTypex_W) + return ret +} + +func (e *emitNative) invokeByteSliceTypex_W(key []byte, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceTypex_W) + return ret +} + +func (e *emitNative) invokeETByteSliceTypex_W(t typex.EventTime, key []byte, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceTypex_X) + return ret +} + +func (e *emitNative) invokeByteSliceTypex_X(key []byte, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceTypex_X) + return ret +} + +func (e *emitNative) invokeETByteSliceTypex_X(t typex.EventTime, key []byte, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceTypex_Y) + return ret +} + +func (e *emitNative) invokeByteSliceTypex_Y(key []byte, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceTypex_Y) + return ret +} + +func (e *emitNative) invokeETByteSliceTypex_Y(t typex.EventTime, key []byte, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerByteSliceTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeByteSliceTypex_Z) + return ret +} + +func (e *emitNative) invokeByteSliceTypex_Z(key []byte, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETByteSliceTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETByteSliceTypex_Z) + return ret +} + +func (e *emitNative) invokeETByteSliceTypex_Z(t typex.EventTime, key []byte, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBool) + return ret +} + +func (e *emitNative) invokeBool(elm bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBool) + return ret +} + +func (e *emitNative) invokeETBool(t typex.EventTime, elm bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolByteSlice) + return ret +} + +func (e *emitNative) invokeBoolByteSlice(key bool, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolByteSlice) + return ret +} + +func (e *emitNative) invokeETBoolByteSlice(t typex.EventTime, key bool, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolBool) + return ret +} + +func (e *emitNative) invokeBoolBool(key bool, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolBool) + return ret +} + +func (e *emitNative) invokeETBoolBool(t typex.EventTime, key bool, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolString) + return ret +} + +func (e *emitNative) invokeBoolString(key bool, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolString) + return ret +} + +func (e *emitNative) invokeETBoolString(t typex.EventTime, key bool, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolInt) + return ret +} + +func (e *emitNative) invokeBoolInt(key bool, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolInt) + return ret +} + +func (e *emitNative) invokeETBoolInt(t typex.EventTime, key bool, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolInt8) + return ret +} + +func (e *emitNative) invokeBoolInt8(key bool, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolInt8) + return ret +} + +func (e *emitNative) invokeETBoolInt8(t typex.EventTime, key bool, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolInt16) + return ret +} + +func (e *emitNative) invokeBoolInt16(key bool, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolInt16) + return ret +} + +func (e *emitNative) invokeETBoolInt16(t typex.EventTime, key bool, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolInt32) + return ret +} + +func (e *emitNative) invokeBoolInt32(key bool, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolInt32) + return ret +} + +func (e *emitNative) invokeETBoolInt32(t typex.EventTime, key bool, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolInt64) + return ret +} + +func (e *emitNative) invokeBoolInt64(key bool, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolInt64) + return ret +} + +func (e *emitNative) invokeETBoolInt64(t typex.EventTime, key bool, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolUint) + return ret +} + +func (e *emitNative) invokeBoolUint(key bool, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolUint) + return ret +} + +func (e *emitNative) invokeETBoolUint(t typex.EventTime, key bool, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolUint8) + return ret +} + +func (e *emitNative) invokeBoolUint8(key bool, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolUint8) + return ret +} + +func (e *emitNative) invokeETBoolUint8(t typex.EventTime, key bool, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolUint16) + return ret +} + +func (e *emitNative) invokeBoolUint16(key bool, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolUint16) + return ret +} + +func (e *emitNative) invokeETBoolUint16(t typex.EventTime, key bool, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolUint32) + return ret +} + +func (e *emitNative) invokeBoolUint32(key bool, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolUint32) + return ret +} + +func (e *emitNative) invokeETBoolUint32(t typex.EventTime, key bool, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolUint64) + return ret +} + +func (e *emitNative) invokeBoolUint64(key bool, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolUint64) + return ret +} + +func (e *emitNative) invokeETBoolUint64(t typex.EventTime, key bool, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolFloat32) + return ret +} + +func (e *emitNative) invokeBoolFloat32(key bool, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolFloat32) + return ret +} + +func (e *emitNative) invokeETBoolFloat32(t typex.EventTime, key bool, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolFloat64) + return ret +} + +func (e *emitNative) invokeBoolFloat64(key bool, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolFloat64) + return ret +} + +func (e *emitNative) invokeETBoolFloat64(t typex.EventTime, key bool, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolTypex_T) + return ret +} + +func (e *emitNative) invokeBoolTypex_T(key bool, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolTypex_T) + return ret +} + +func (e *emitNative) invokeETBoolTypex_T(t typex.EventTime, key bool, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolTypex_U) + return ret +} + +func (e *emitNative) invokeBoolTypex_U(key bool, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolTypex_U) + return ret +} + +func (e *emitNative) invokeETBoolTypex_U(t typex.EventTime, key bool, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolTypex_V) + return ret +} + +func (e *emitNative) invokeBoolTypex_V(key bool, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolTypex_V) + return ret +} + +func (e *emitNative) invokeETBoolTypex_V(t typex.EventTime, key bool, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolTypex_W) + return ret +} + +func (e *emitNative) invokeBoolTypex_W(key bool, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolTypex_W) + return ret +} + +func (e *emitNative) invokeETBoolTypex_W(t typex.EventTime, key bool, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolTypex_X) + return ret +} + +func (e *emitNative) invokeBoolTypex_X(key bool, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolTypex_X) + return ret +} + +func (e *emitNative) invokeETBoolTypex_X(t typex.EventTime, key bool, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolTypex_Y) + return ret +} + +func (e *emitNative) invokeBoolTypex_Y(key bool, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolTypex_Y) + return ret +} + +func (e *emitNative) invokeETBoolTypex_Y(t typex.EventTime, key bool, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerBoolTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeBoolTypex_Z) + return ret +} + +func (e *emitNative) invokeBoolTypex_Z(key bool, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETBoolTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETBoolTypex_Z) + return ret +} + +func (e *emitNative) invokeETBoolTypex_Z(t typex.EventTime, key bool, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeString) + return ret +} + +func (e *emitNative) invokeString(elm string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETString) + return ret +} + +func (e *emitNative) invokeETString(t typex.EventTime, elm string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringByteSlice) + return ret +} + +func (e *emitNative) invokeStringByteSlice(key string, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringByteSlice) + return ret +} + +func (e *emitNative) invokeETStringByteSlice(t typex.EventTime, key string, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringBool) + return ret +} + +func (e *emitNative) invokeStringBool(key string, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringBool) + return ret +} + +func (e *emitNative) invokeETStringBool(t typex.EventTime, key string, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringString) + return ret +} + +func (e *emitNative) invokeStringString(key string, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringString) + return ret +} + +func (e *emitNative) invokeETStringString(t typex.EventTime, key string, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringInt) + return ret +} + +func (e *emitNative) invokeStringInt(key string, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringInt) + return ret +} + +func (e *emitNative) invokeETStringInt(t typex.EventTime, key string, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringInt8) + return ret +} + +func (e *emitNative) invokeStringInt8(key string, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringInt8) + return ret +} + +func (e *emitNative) invokeETStringInt8(t typex.EventTime, key string, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringInt16) + return ret +} + +func (e *emitNative) invokeStringInt16(key string, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringInt16) + return ret +} + +func (e *emitNative) invokeETStringInt16(t typex.EventTime, key string, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringInt32) + return ret +} + +func (e *emitNative) invokeStringInt32(key string, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringInt32) + return ret +} + +func (e *emitNative) invokeETStringInt32(t typex.EventTime, key string, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringInt64) + return ret +} + +func (e *emitNative) invokeStringInt64(key string, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringInt64) + return ret +} + +func (e *emitNative) invokeETStringInt64(t typex.EventTime, key string, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringUint) + return ret +} + +func (e *emitNative) invokeStringUint(key string, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringUint) + return ret +} + +func (e *emitNative) invokeETStringUint(t typex.EventTime, key string, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringUint8) + return ret +} + +func (e *emitNative) invokeStringUint8(key string, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringUint8) + return ret +} + +func (e *emitNative) invokeETStringUint8(t typex.EventTime, key string, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringUint16) + return ret +} + +func (e *emitNative) invokeStringUint16(key string, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringUint16) + return ret +} + +func (e *emitNative) invokeETStringUint16(t typex.EventTime, key string, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringUint32) + return ret +} + +func (e *emitNative) invokeStringUint32(key string, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringUint32) + return ret +} + +func (e *emitNative) invokeETStringUint32(t typex.EventTime, key string, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringUint64) + return ret +} + +func (e *emitNative) invokeStringUint64(key string, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringUint64) + return ret +} + +func (e *emitNative) invokeETStringUint64(t typex.EventTime, key string, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringFloat32) + return ret +} + +func (e *emitNative) invokeStringFloat32(key string, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringFloat32) + return ret +} + +func (e *emitNative) invokeETStringFloat32(t typex.EventTime, key string, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringFloat64) + return ret +} + +func (e *emitNative) invokeStringFloat64(key string, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringFloat64) + return ret +} + +func (e *emitNative) invokeETStringFloat64(t typex.EventTime, key string, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringTypex_T) + return ret +} + +func (e *emitNative) invokeStringTypex_T(key string, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringTypex_T) + return ret +} + +func (e *emitNative) invokeETStringTypex_T(t typex.EventTime, key string, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringTypex_U) + return ret +} + +func (e *emitNative) invokeStringTypex_U(key string, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringTypex_U) + return ret +} + +func (e *emitNative) invokeETStringTypex_U(t typex.EventTime, key string, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringTypex_V) + return ret +} + +func (e *emitNative) invokeStringTypex_V(key string, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringTypex_V) + return ret +} + +func (e *emitNative) invokeETStringTypex_V(t typex.EventTime, key string, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringTypex_W) + return ret +} + +func (e *emitNative) invokeStringTypex_W(key string, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringTypex_W) + return ret +} + +func (e *emitNative) invokeETStringTypex_W(t typex.EventTime, key string, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringTypex_X) + return ret +} + +func (e *emitNative) invokeStringTypex_X(key string, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringTypex_X) + return ret +} + +func (e *emitNative) invokeETStringTypex_X(t typex.EventTime, key string, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringTypex_Y) + return ret +} + +func (e *emitNative) invokeStringTypex_Y(key string, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringTypex_Y) + return ret +} + +func (e *emitNative) invokeETStringTypex_Y(t typex.EventTime, key string, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerStringTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeStringTypex_Z) + return ret +} + +func (e *emitNative) invokeStringTypex_Z(key string, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETStringTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETStringTypex_Z) + return ret +} + +func (e *emitNative) invokeETStringTypex_Z(t typex.EventTime, key string, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt) + return ret +} + +func (e *emitNative) invokeInt(elm int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt) + return ret +} + +func (e *emitNative) invokeETInt(t typex.EventTime, elm int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntByteSlice) + return ret +} + +func (e *emitNative) invokeIntByteSlice(key int, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntByteSlice) + return ret +} + +func (e *emitNative) invokeETIntByteSlice(t typex.EventTime, key int, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntBool) + return ret +} + +func (e *emitNative) invokeIntBool(key int, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntBool) + return ret +} + +func (e *emitNative) invokeETIntBool(t typex.EventTime, key int, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntString) + return ret +} + +func (e *emitNative) invokeIntString(key int, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntString) + return ret +} + +func (e *emitNative) invokeETIntString(t typex.EventTime, key int, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntInt) + return ret +} + +func (e *emitNative) invokeIntInt(key int, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntInt) + return ret +} + +func (e *emitNative) invokeETIntInt(t typex.EventTime, key int, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntInt8) + return ret +} + +func (e *emitNative) invokeIntInt8(key int, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntInt8) + return ret +} + +func (e *emitNative) invokeETIntInt8(t typex.EventTime, key int, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntInt16) + return ret +} + +func (e *emitNative) invokeIntInt16(key int, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntInt16) + return ret +} + +func (e *emitNative) invokeETIntInt16(t typex.EventTime, key int, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntInt32) + return ret +} + +func (e *emitNative) invokeIntInt32(key int, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntInt32) + return ret +} + +func (e *emitNative) invokeETIntInt32(t typex.EventTime, key int, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntInt64) + return ret +} + +func (e *emitNative) invokeIntInt64(key int, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntInt64) + return ret +} + +func (e *emitNative) invokeETIntInt64(t typex.EventTime, key int, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntUint) + return ret +} + +func (e *emitNative) invokeIntUint(key int, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntUint) + return ret +} + +func (e *emitNative) invokeETIntUint(t typex.EventTime, key int, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntUint8) + return ret +} + +func (e *emitNative) invokeIntUint8(key int, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntUint8) + return ret +} + +func (e *emitNative) invokeETIntUint8(t typex.EventTime, key int, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntUint16) + return ret +} + +func (e *emitNative) invokeIntUint16(key int, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntUint16) + return ret +} + +func (e *emitNative) invokeETIntUint16(t typex.EventTime, key int, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntUint32) + return ret +} + +func (e *emitNative) invokeIntUint32(key int, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntUint32) + return ret +} + +func (e *emitNative) invokeETIntUint32(t typex.EventTime, key int, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntUint64) + return ret +} + +func (e *emitNative) invokeIntUint64(key int, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntUint64) + return ret +} + +func (e *emitNative) invokeETIntUint64(t typex.EventTime, key int, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntFloat32) + return ret +} + +func (e *emitNative) invokeIntFloat32(key int, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntFloat32) + return ret +} + +func (e *emitNative) invokeETIntFloat32(t typex.EventTime, key int, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntFloat64) + return ret +} + +func (e *emitNative) invokeIntFloat64(key int, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntFloat64) + return ret +} + +func (e *emitNative) invokeETIntFloat64(t typex.EventTime, key int, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntTypex_T) + return ret +} + +func (e *emitNative) invokeIntTypex_T(key int, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntTypex_T) + return ret +} + +func (e *emitNative) invokeETIntTypex_T(t typex.EventTime, key int, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntTypex_U) + return ret +} + +func (e *emitNative) invokeIntTypex_U(key int, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntTypex_U) + return ret +} + +func (e *emitNative) invokeETIntTypex_U(t typex.EventTime, key int, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntTypex_V) + return ret +} + +func (e *emitNative) invokeIntTypex_V(key int, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntTypex_V) + return ret +} + +func (e *emitNative) invokeETIntTypex_V(t typex.EventTime, key int, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntTypex_W) + return ret +} + +func (e *emitNative) invokeIntTypex_W(key int, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntTypex_W) + return ret +} + +func (e *emitNative) invokeETIntTypex_W(t typex.EventTime, key int, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntTypex_X) + return ret +} + +func (e *emitNative) invokeIntTypex_X(key int, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntTypex_X) + return ret +} + +func (e *emitNative) invokeETIntTypex_X(t typex.EventTime, key int, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntTypex_Y) + return ret +} + +func (e *emitNative) invokeIntTypex_Y(key int, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntTypex_Y) + return ret +} + +func (e *emitNative) invokeETIntTypex_Y(t typex.EventTime, key int, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerIntTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeIntTypex_Z) + return ret +} + +func (e *emitNative) invokeIntTypex_Z(key int, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETIntTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETIntTypex_Z) + return ret +} + +func (e *emitNative) invokeETIntTypex_Z(t typex.EventTime, key int, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8) + return ret +} + +func (e *emitNative) invokeInt8(elm int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8) + return ret +} + +func (e *emitNative) invokeETInt8(t typex.EventTime, elm int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8ByteSlice) + return ret +} + +func (e *emitNative) invokeInt8ByteSlice(key int8, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8ByteSlice) + return ret +} + +func (e *emitNative) invokeETInt8ByteSlice(t typex.EventTime, key int8, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Bool) + return ret +} + +func (e *emitNative) invokeInt8Bool(key int8, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Bool) + return ret +} + +func (e *emitNative) invokeETInt8Bool(t typex.EventTime, key int8, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8String) + return ret +} + +func (e *emitNative) invokeInt8String(key int8, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8String) + return ret +} + +func (e *emitNative) invokeETInt8String(t typex.EventTime, key int8, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Int) + return ret +} + +func (e *emitNative) invokeInt8Int(key int8, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Int) + return ret +} + +func (e *emitNative) invokeETInt8Int(t typex.EventTime, key int8, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Int8) + return ret +} + +func (e *emitNative) invokeInt8Int8(key int8, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Int8) + return ret +} + +func (e *emitNative) invokeETInt8Int8(t typex.EventTime, key int8, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Int16) + return ret +} + +func (e *emitNative) invokeInt8Int16(key int8, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Int16) + return ret +} + +func (e *emitNative) invokeETInt8Int16(t typex.EventTime, key int8, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Int32) + return ret +} + +func (e *emitNative) invokeInt8Int32(key int8, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Int32) + return ret +} + +func (e *emitNative) invokeETInt8Int32(t typex.EventTime, key int8, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Int64) + return ret +} + +func (e *emitNative) invokeInt8Int64(key int8, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Int64) + return ret +} + +func (e *emitNative) invokeETInt8Int64(t typex.EventTime, key int8, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Uint) + return ret +} + +func (e *emitNative) invokeInt8Uint(key int8, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Uint) + return ret +} + +func (e *emitNative) invokeETInt8Uint(t typex.EventTime, key int8, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Uint8) + return ret +} + +func (e *emitNative) invokeInt8Uint8(key int8, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Uint8) + return ret +} + +func (e *emitNative) invokeETInt8Uint8(t typex.EventTime, key int8, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Uint16) + return ret +} + +func (e *emitNative) invokeInt8Uint16(key int8, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Uint16) + return ret +} + +func (e *emitNative) invokeETInt8Uint16(t typex.EventTime, key int8, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Uint32) + return ret +} + +func (e *emitNative) invokeInt8Uint32(key int8, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Uint32) + return ret +} + +func (e *emitNative) invokeETInt8Uint32(t typex.EventTime, key int8, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Uint64) + return ret +} + +func (e *emitNative) invokeInt8Uint64(key int8, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Uint64) + return ret +} + +func (e *emitNative) invokeETInt8Uint64(t typex.EventTime, key int8, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Float32) + return ret +} + +func (e *emitNative) invokeInt8Float32(key int8, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Float32) + return ret +} + +func (e *emitNative) invokeETInt8Float32(t typex.EventTime, key int8, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Float64) + return ret +} + +func (e *emitNative) invokeInt8Float64(key int8, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Float64) + return ret +} + +func (e *emitNative) invokeETInt8Float64(t typex.EventTime, key int8, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Typex_T) + return ret +} + +func (e *emitNative) invokeInt8Typex_T(key int8, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Typex_T) + return ret +} + +func (e *emitNative) invokeETInt8Typex_T(t typex.EventTime, key int8, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Typex_U) + return ret +} + +func (e *emitNative) invokeInt8Typex_U(key int8, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Typex_U) + return ret +} + +func (e *emitNative) invokeETInt8Typex_U(t typex.EventTime, key int8, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Typex_V) + return ret +} + +func (e *emitNative) invokeInt8Typex_V(key int8, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Typex_V) + return ret +} + +func (e *emitNative) invokeETInt8Typex_V(t typex.EventTime, key int8, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Typex_W) + return ret +} + +func (e *emitNative) invokeInt8Typex_W(key int8, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Typex_W) + return ret +} + +func (e *emitNative) invokeETInt8Typex_W(t typex.EventTime, key int8, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Typex_X) + return ret +} + +func (e *emitNative) invokeInt8Typex_X(key int8, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Typex_X) + return ret +} + +func (e *emitNative) invokeETInt8Typex_X(t typex.EventTime, key int8, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Typex_Y) + return ret +} + +func (e *emitNative) invokeInt8Typex_Y(key int8, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Typex_Y) + return ret +} + +func (e *emitNative) invokeETInt8Typex_Y(t typex.EventTime, key int8, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt8Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt8Typex_Z) + return ret +} + +func (e *emitNative) invokeInt8Typex_Z(key int8, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt8Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt8Typex_Z) + return ret +} + +func (e *emitNative) invokeETInt8Typex_Z(t typex.EventTime, key int8, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16) + return ret +} + +func (e *emitNative) invokeInt16(elm int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16) + return ret +} + +func (e *emitNative) invokeETInt16(t typex.EventTime, elm int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16ByteSlice) + return ret +} + +func (e *emitNative) invokeInt16ByteSlice(key int16, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16ByteSlice) + return ret +} + +func (e *emitNative) invokeETInt16ByteSlice(t typex.EventTime, key int16, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Bool) + return ret +} + +func (e *emitNative) invokeInt16Bool(key int16, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Bool) + return ret +} + +func (e *emitNative) invokeETInt16Bool(t typex.EventTime, key int16, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16String) + return ret +} + +func (e *emitNative) invokeInt16String(key int16, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16String) + return ret +} + +func (e *emitNative) invokeETInt16String(t typex.EventTime, key int16, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Int) + return ret +} + +func (e *emitNative) invokeInt16Int(key int16, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Int) + return ret +} + +func (e *emitNative) invokeETInt16Int(t typex.EventTime, key int16, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Int8) + return ret +} + +func (e *emitNative) invokeInt16Int8(key int16, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Int8) + return ret +} + +func (e *emitNative) invokeETInt16Int8(t typex.EventTime, key int16, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Int16) + return ret +} + +func (e *emitNative) invokeInt16Int16(key int16, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Int16) + return ret +} + +func (e *emitNative) invokeETInt16Int16(t typex.EventTime, key int16, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Int32) + return ret +} + +func (e *emitNative) invokeInt16Int32(key int16, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Int32) + return ret +} + +func (e *emitNative) invokeETInt16Int32(t typex.EventTime, key int16, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Int64) + return ret +} + +func (e *emitNative) invokeInt16Int64(key int16, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Int64) + return ret +} + +func (e *emitNative) invokeETInt16Int64(t typex.EventTime, key int16, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Uint) + return ret +} + +func (e *emitNative) invokeInt16Uint(key int16, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Uint) + return ret +} + +func (e *emitNative) invokeETInt16Uint(t typex.EventTime, key int16, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Uint8) + return ret +} + +func (e *emitNative) invokeInt16Uint8(key int16, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Uint8) + return ret +} + +func (e *emitNative) invokeETInt16Uint8(t typex.EventTime, key int16, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Uint16) + return ret +} + +func (e *emitNative) invokeInt16Uint16(key int16, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Uint16) + return ret +} + +func (e *emitNative) invokeETInt16Uint16(t typex.EventTime, key int16, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Uint32) + return ret +} + +func (e *emitNative) invokeInt16Uint32(key int16, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Uint32) + return ret +} + +func (e *emitNative) invokeETInt16Uint32(t typex.EventTime, key int16, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Uint64) + return ret +} + +func (e *emitNative) invokeInt16Uint64(key int16, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Uint64) + return ret +} + +func (e *emitNative) invokeETInt16Uint64(t typex.EventTime, key int16, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Float32) + return ret +} + +func (e *emitNative) invokeInt16Float32(key int16, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Float32) + return ret +} + +func (e *emitNative) invokeETInt16Float32(t typex.EventTime, key int16, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Float64) + return ret +} + +func (e *emitNative) invokeInt16Float64(key int16, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Float64) + return ret +} + +func (e *emitNative) invokeETInt16Float64(t typex.EventTime, key int16, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Typex_T) + return ret +} + +func (e *emitNative) invokeInt16Typex_T(key int16, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Typex_T) + return ret +} + +func (e *emitNative) invokeETInt16Typex_T(t typex.EventTime, key int16, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Typex_U) + return ret +} + +func (e *emitNative) invokeInt16Typex_U(key int16, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Typex_U) + return ret +} + +func (e *emitNative) invokeETInt16Typex_U(t typex.EventTime, key int16, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Typex_V) + return ret +} + +func (e *emitNative) invokeInt16Typex_V(key int16, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Typex_V) + return ret +} + +func (e *emitNative) invokeETInt16Typex_V(t typex.EventTime, key int16, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Typex_W) + return ret +} + +func (e *emitNative) invokeInt16Typex_W(key int16, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Typex_W) + return ret +} + +func (e *emitNative) invokeETInt16Typex_W(t typex.EventTime, key int16, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Typex_X) + return ret +} + +func (e *emitNative) invokeInt16Typex_X(key int16, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Typex_X) + return ret +} + +func (e *emitNative) invokeETInt16Typex_X(t typex.EventTime, key int16, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Typex_Y) + return ret +} + +func (e *emitNative) invokeInt16Typex_Y(key int16, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Typex_Y) + return ret +} + +func (e *emitNative) invokeETInt16Typex_Y(t typex.EventTime, key int16, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt16Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt16Typex_Z) + return ret +} + +func (e *emitNative) invokeInt16Typex_Z(key int16, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt16Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt16Typex_Z) + return ret +} + +func (e *emitNative) invokeETInt16Typex_Z(t typex.EventTime, key int16, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32) + return ret +} + +func (e *emitNative) invokeInt32(elm int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32) + return ret +} + +func (e *emitNative) invokeETInt32(t typex.EventTime, elm int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32ByteSlice) + return ret +} + +func (e *emitNative) invokeInt32ByteSlice(key int32, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32ByteSlice) + return ret +} + +func (e *emitNative) invokeETInt32ByteSlice(t typex.EventTime, key int32, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Bool) + return ret +} + +func (e *emitNative) invokeInt32Bool(key int32, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Bool) + return ret +} + +func (e *emitNative) invokeETInt32Bool(t typex.EventTime, key int32, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32String) + return ret +} + +func (e *emitNative) invokeInt32String(key int32, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32String) + return ret +} + +func (e *emitNative) invokeETInt32String(t typex.EventTime, key int32, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Int) + return ret +} + +func (e *emitNative) invokeInt32Int(key int32, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Int) + return ret +} + +func (e *emitNative) invokeETInt32Int(t typex.EventTime, key int32, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Int8) + return ret +} + +func (e *emitNative) invokeInt32Int8(key int32, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Int8) + return ret +} + +func (e *emitNative) invokeETInt32Int8(t typex.EventTime, key int32, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Int16) + return ret +} + +func (e *emitNative) invokeInt32Int16(key int32, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Int16) + return ret +} + +func (e *emitNative) invokeETInt32Int16(t typex.EventTime, key int32, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Int32) + return ret +} + +func (e *emitNative) invokeInt32Int32(key int32, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Int32) + return ret +} + +func (e *emitNative) invokeETInt32Int32(t typex.EventTime, key int32, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Int64) + return ret +} + +func (e *emitNative) invokeInt32Int64(key int32, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Int64) + return ret +} + +func (e *emitNative) invokeETInt32Int64(t typex.EventTime, key int32, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Uint) + return ret +} + +func (e *emitNative) invokeInt32Uint(key int32, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Uint) + return ret +} + +func (e *emitNative) invokeETInt32Uint(t typex.EventTime, key int32, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Uint8) + return ret +} + +func (e *emitNative) invokeInt32Uint8(key int32, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Uint8) + return ret +} + +func (e *emitNative) invokeETInt32Uint8(t typex.EventTime, key int32, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Uint16) + return ret +} + +func (e *emitNative) invokeInt32Uint16(key int32, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Uint16) + return ret +} + +func (e *emitNative) invokeETInt32Uint16(t typex.EventTime, key int32, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Uint32) + return ret +} + +func (e *emitNative) invokeInt32Uint32(key int32, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Uint32) + return ret +} + +func (e *emitNative) invokeETInt32Uint32(t typex.EventTime, key int32, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Uint64) + return ret +} + +func (e *emitNative) invokeInt32Uint64(key int32, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Uint64) + return ret +} + +func (e *emitNative) invokeETInt32Uint64(t typex.EventTime, key int32, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Float32) + return ret +} + +func (e *emitNative) invokeInt32Float32(key int32, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Float32) + return ret +} + +func (e *emitNative) invokeETInt32Float32(t typex.EventTime, key int32, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Float64) + return ret +} + +func (e *emitNative) invokeInt32Float64(key int32, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Float64) + return ret +} + +func (e *emitNative) invokeETInt32Float64(t typex.EventTime, key int32, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Typex_T) + return ret +} + +func (e *emitNative) invokeInt32Typex_T(key int32, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Typex_T) + return ret +} + +func (e *emitNative) invokeETInt32Typex_T(t typex.EventTime, key int32, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Typex_U) + return ret +} + +func (e *emitNative) invokeInt32Typex_U(key int32, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Typex_U) + return ret +} + +func (e *emitNative) invokeETInt32Typex_U(t typex.EventTime, key int32, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Typex_V) + return ret +} + +func (e *emitNative) invokeInt32Typex_V(key int32, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Typex_V) + return ret +} + +func (e *emitNative) invokeETInt32Typex_V(t typex.EventTime, key int32, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Typex_W) + return ret +} + +func (e *emitNative) invokeInt32Typex_W(key int32, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Typex_W) + return ret +} + +func (e *emitNative) invokeETInt32Typex_W(t typex.EventTime, key int32, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Typex_X) + return ret +} + +func (e *emitNative) invokeInt32Typex_X(key int32, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Typex_X) + return ret +} + +func (e *emitNative) invokeETInt32Typex_X(t typex.EventTime, key int32, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Typex_Y) + return ret +} + +func (e *emitNative) invokeInt32Typex_Y(key int32, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Typex_Y) + return ret +} + +func (e *emitNative) invokeETInt32Typex_Y(t typex.EventTime, key int32, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt32Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt32Typex_Z) + return ret +} + +func (e *emitNative) invokeInt32Typex_Z(key int32, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt32Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt32Typex_Z) + return ret +} + +func (e *emitNative) invokeETInt32Typex_Z(t typex.EventTime, key int32, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64) + return ret +} + +func (e *emitNative) invokeInt64(elm int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64) + return ret +} + +func (e *emitNative) invokeETInt64(t typex.EventTime, elm int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64ByteSlice) + return ret +} + +func (e *emitNative) invokeInt64ByteSlice(key int64, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64ByteSlice) + return ret +} + +func (e *emitNative) invokeETInt64ByteSlice(t typex.EventTime, key int64, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Bool) + return ret +} + +func (e *emitNative) invokeInt64Bool(key int64, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Bool) + return ret +} + +func (e *emitNative) invokeETInt64Bool(t typex.EventTime, key int64, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64String) + return ret +} + +func (e *emitNative) invokeInt64String(key int64, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64String) + return ret +} + +func (e *emitNative) invokeETInt64String(t typex.EventTime, key int64, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Int) + return ret +} + +func (e *emitNative) invokeInt64Int(key int64, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Int) + return ret +} + +func (e *emitNative) invokeETInt64Int(t typex.EventTime, key int64, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Int8) + return ret +} + +func (e *emitNative) invokeInt64Int8(key int64, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Int8) + return ret +} + +func (e *emitNative) invokeETInt64Int8(t typex.EventTime, key int64, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Int16) + return ret +} + +func (e *emitNative) invokeInt64Int16(key int64, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Int16) + return ret +} + +func (e *emitNative) invokeETInt64Int16(t typex.EventTime, key int64, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Int32) + return ret +} + +func (e *emitNative) invokeInt64Int32(key int64, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Int32) + return ret +} + +func (e *emitNative) invokeETInt64Int32(t typex.EventTime, key int64, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Int64) + return ret +} + +func (e *emitNative) invokeInt64Int64(key int64, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Int64) + return ret +} + +func (e *emitNative) invokeETInt64Int64(t typex.EventTime, key int64, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Uint) + return ret +} + +func (e *emitNative) invokeInt64Uint(key int64, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Uint) + return ret +} + +func (e *emitNative) invokeETInt64Uint(t typex.EventTime, key int64, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Uint8) + return ret +} + +func (e *emitNative) invokeInt64Uint8(key int64, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Uint8) + return ret +} + +func (e *emitNative) invokeETInt64Uint8(t typex.EventTime, key int64, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Uint16) + return ret +} + +func (e *emitNative) invokeInt64Uint16(key int64, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Uint16) + return ret +} + +func (e *emitNative) invokeETInt64Uint16(t typex.EventTime, key int64, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Uint32) + return ret +} + +func (e *emitNative) invokeInt64Uint32(key int64, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Uint32) + return ret +} + +func (e *emitNative) invokeETInt64Uint32(t typex.EventTime, key int64, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Uint64) + return ret +} + +func (e *emitNative) invokeInt64Uint64(key int64, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Uint64) + return ret +} + +func (e *emitNative) invokeETInt64Uint64(t typex.EventTime, key int64, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Float32) + return ret +} + +func (e *emitNative) invokeInt64Float32(key int64, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Float32) + return ret +} + +func (e *emitNative) invokeETInt64Float32(t typex.EventTime, key int64, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Float64) + return ret +} + +func (e *emitNative) invokeInt64Float64(key int64, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Float64) + return ret +} + +func (e *emitNative) invokeETInt64Float64(t typex.EventTime, key int64, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Typex_T) + return ret +} + +func (e *emitNative) invokeInt64Typex_T(key int64, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Typex_T) + return ret +} + +func (e *emitNative) invokeETInt64Typex_T(t typex.EventTime, key int64, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Typex_U) + return ret +} + +func (e *emitNative) invokeInt64Typex_U(key int64, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Typex_U) + return ret +} + +func (e *emitNative) invokeETInt64Typex_U(t typex.EventTime, key int64, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Typex_V) + return ret +} + +func (e *emitNative) invokeInt64Typex_V(key int64, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Typex_V) + return ret +} + +func (e *emitNative) invokeETInt64Typex_V(t typex.EventTime, key int64, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Typex_W) + return ret +} + +func (e *emitNative) invokeInt64Typex_W(key int64, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Typex_W) + return ret +} + +func (e *emitNative) invokeETInt64Typex_W(t typex.EventTime, key int64, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Typex_X) + return ret +} + +func (e *emitNative) invokeInt64Typex_X(key int64, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Typex_X) + return ret +} + +func (e *emitNative) invokeETInt64Typex_X(t typex.EventTime, key int64, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Typex_Y) + return ret +} + +func (e *emitNative) invokeInt64Typex_Y(key int64, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Typex_Y) + return ret +} + +func (e *emitNative) invokeETInt64Typex_Y(t typex.EventTime, key int64, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerInt64Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeInt64Typex_Z) + return ret +} + +func (e *emitNative) invokeInt64Typex_Z(key int64, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETInt64Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETInt64Typex_Z) + return ret +} + +func (e *emitNative) invokeETInt64Typex_Z(t typex.EventTime, key int64, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint) + return ret +} + +func (e *emitNative) invokeUint(elm uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint) + return ret +} + +func (e *emitNative) invokeETUint(t typex.EventTime, elm uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintByteSlice) + return ret +} + +func (e *emitNative) invokeUintByteSlice(key uint, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintByteSlice) + return ret +} + +func (e *emitNative) invokeETUintByteSlice(t typex.EventTime, key uint, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintBool) + return ret +} + +func (e *emitNative) invokeUintBool(key uint, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintBool) + return ret +} + +func (e *emitNative) invokeETUintBool(t typex.EventTime, key uint, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintString) + return ret +} + +func (e *emitNative) invokeUintString(key uint, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintString) + return ret +} + +func (e *emitNative) invokeETUintString(t typex.EventTime, key uint, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintInt) + return ret +} + +func (e *emitNative) invokeUintInt(key uint, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintInt) + return ret +} + +func (e *emitNative) invokeETUintInt(t typex.EventTime, key uint, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintInt8) + return ret +} + +func (e *emitNative) invokeUintInt8(key uint, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintInt8) + return ret +} + +func (e *emitNative) invokeETUintInt8(t typex.EventTime, key uint, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintInt16) + return ret +} + +func (e *emitNative) invokeUintInt16(key uint, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintInt16) + return ret +} + +func (e *emitNative) invokeETUintInt16(t typex.EventTime, key uint, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintInt32) + return ret +} + +func (e *emitNative) invokeUintInt32(key uint, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintInt32) + return ret +} + +func (e *emitNative) invokeETUintInt32(t typex.EventTime, key uint, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintInt64) + return ret +} + +func (e *emitNative) invokeUintInt64(key uint, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintInt64) + return ret +} + +func (e *emitNative) invokeETUintInt64(t typex.EventTime, key uint, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintUint) + return ret +} + +func (e *emitNative) invokeUintUint(key uint, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintUint) + return ret +} + +func (e *emitNative) invokeETUintUint(t typex.EventTime, key uint, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintUint8) + return ret +} + +func (e *emitNative) invokeUintUint8(key uint, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintUint8) + return ret +} + +func (e *emitNative) invokeETUintUint8(t typex.EventTime, key uint, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintUint16) + return ret +} + +func (e *emitNative) invokeUintUint16(key uint, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintUint16) + return ret +} + +func (e *emitNative) invokeETUintUint16(t typex.EventTime, key uint, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintUint32) + return ret +} + +func (e *emitNative) invokeUintUint32(key uint, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintUint32) + return ret +} + +func (e *emitNative) invokeETUintUint32(t typex.EventTime, key uint, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintUint64) + return ret +} + +func (e *emitNative) invokeUintUint64(key uint, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintUint64) + return ret +} + +func (e *emitNative) invokeETUintUint64(t typex.EventTime, key uint, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintFloat32) + return ret +} + +func (e *emitNative) invokeUintFloat32(key uint, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintFloat32) + return ret +} + +func (e *emitNative) invokeETUintFloat32(t typex.EventTime, key uint, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintFloat64) + return ret +} + +func (e *emitNative) invokeUintFloat64(key uint, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintFloat64) + return ret +} + +func (e *emitNative) invokeETUintFloat64(t typex.EventTime, key uint, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintTypex_T) + return ret +} + +func (e *emitNative) invokeUintTypex_T(key uint, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintTypex_T) + return ret +} + +func (e *emitNative) invokeETUintTypex_T(t typex.EventTime, key uint, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintTypex_U) + return ret +} + +func (e *emitNative) invokeUintTypex_U(key uint, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintTypex_U) + return ret +} + +func (e *emitNative) invokeETUintTypex_U(t typex.EventTime, key uint, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintTypex_V) + return ret +} + +func (e *emitNative) invokeUintTypex_V(key uint, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintTypex_V) + return ret +} + +func (e *emitNative) invokeETUintTypex_V(t typex.EventTime, key uint, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintTypex_W) + return ret +} + +func (e *emitNative) invokeUintTypex_W(key uint, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintTypex_W) + return ret +} + +func (e *emitNative) invokeETUintTypex_W(t typex.EventTime, key uint, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintTypex_X) + return ret +} + +func (e *emitNative) invokeUintTypex_X(key uint, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintTypex_X) + return ret +} + +func (e *emitNative) invokeETUintTypex_X(t typex.EventTime, key uint, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintTypex_Y) + return ret +} + +func (e *emitNative) invokeUintTypex_Y(key uint, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintTypex_Y) + return ret +} + +func (e *emitNative) invokeETUintTypex_Y(t typex.EventTime, key uint, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUintTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUintTypex_Z) + return ret +} + +func (e *emitNative) invokeUintTypex_Z(key uint, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUintTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUintTypex_Z) + return ret +} + +func (e *emitNative) invokeETUintTypex_Z(t typex.EventTime, key uint, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8) + return ret +} + +func (e *emitNative) invokeUint8(elm uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8) + return ret +} + +func (e *emitNative) invokeETUint8(t typex.EventTime, elm uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8ByteSlice) + return ret +} + +func (e *emitNative) invokeUint8ByteSlice(key uint8, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8ByteSlice) + return ret +} + +func (e *emitNative) invokeETUint8ByteSlice(t typex.EventTime, key uint8, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Bool) + return ret +} + +func (e *emitNative) invokeUint8Bool(key uint8, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Bool) + return ret +} + +func (e *emitNative) invokeETUint8Bool(t typex.EventTime, key uint8, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8String) + return ret +} + +func (e *emitNative) invokeUint8String(key uint8, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8String) + return ret +} + +func (e *emitNative) invokeETUint8String(t typex.EventTime, key uint8, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Int) + return ret +} + +func (e *emitNative) invokeUint8Int(key uint8, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Int) + return ret +} + +func (e *emitNative) invokeETUint8Int(t typex.EventTime, key uint8, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Int8) + return ret +} + +func (e *emitNative) invokeUint8Int8(key uint8, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Int8) + return ret +} + +func (e *emitNative) invokeETUint8Int8(t typex.EventTime, key uint8, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Int16) + return ret +} + +func (e *emitNative) invokeUint8Int16(key uint8, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Int16) + return ret +} + +func (e *emitNative) invokeETUint8Int16(t typex.EventTime, key uint8, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Int32) + return ret +} + +func (e *emitNative) invokeUint8Int32(key uint8, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Int32) + return ret +} + +func (e *emitNative) invokeETUint8Int32(t typex.EventTime, key uint8, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Int64) + return ret +} + +func (e *emitNative) invokeUint8Int64(key uint8, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Int64) + return ret +} + +func (e *emitNative) invokeETUint8Int64(t typex.EventTime, key uint8, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Uint) + return ret +} + +func (e *emitNative) invokeUint8Uint(key uint8, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Uint) + return ret +} + +func (e *emitNative) invokeETUint8Uint(t typex.EventTime, key uint8, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Uint8) + return ret +} + +func (e *emitNative) invokeUint8Uint8(key uint8, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Uint8) + return ret +} + +func (e *emitNative) invokeETUint8Uint8(t typex.EventTime, key uint8, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Uint16) + return ret +} + +func (e *emitNative) invokeUint8Uint16(key uint8, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Uint16) + return ret +} + +func (e *emitNative) invokeETUint8Uint16(t typex.EventTime, key uint8, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Uint32) + return ret +} + +func (e *emitNative) invokeUint8Uint32(key uint8, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Uint32) + return ret +} + +func (e *emitNative) invokeETUint8Uint32(t typex.EventTime, key uint8, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Uint64) + return ret +} + +func (e *emitNative) invokeUint8Uint64(key uint8, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Uint64) + return ret +} + +func (e *emitNative) invokeETUint8Uint64(t typex.EventTime, key uint8, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Float32) + return ret +} + +func (e *emitNative) invokeUint8Float32(key uint8, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Float32) + return ret +} + +func (e *emitNative) invokeETUint8Float32(t typex.EventTime, key uint8, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Float64) + return ret +} + +func (e *emitNative) invokeUint8Float64(key uint8, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Float64) + return ret +} + +func (e *emitNative) invokeETUint8Float64(t typex.EventTime, key uint8, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Typex_T) + return ret +} + +func (e *emitNative) invokeUint8Typex_T(key uint8, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Typex_T) + return ret +} + +func (e *emitNative) invokeETUint8Typex_T(t typex.EventTime, key uint8, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Typex_U) + return ret +} + +func (e *emitNative) invokeUint8Typex_U(key uint8, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Typex_U) + return ret +} + +func (e *emitNative) invokeETUint8Typex_U(t typex.EventTime, key uint8, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Typex_V) + return ret +} + +func (e *emitNative) invokeUint8Typex_V(key uint8, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Typex_V) + return ret +} + +func (e *emitNative) invokeETUint8Typex_V(t typex.EventTime, key uint8, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Typex_W) + return ret +} + +func (e *emitNative) invokeUint8Typex_W(key uint8, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Typex_W) + return ret +} + +func (e *emitNative) invokeETUint8Typex_W(t typex.EventTime, key uint8, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Typex_X) + return ret +} + +func (e *emitNative) invokeUint8Typex_X(key uint8, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Typex_X) + return ret +} + +func (e *emitNative) invokeETUint8Typex_X(t typex.EventTime, key uint8, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Typex_Y) + return ret +} + +func (e *emitNative) invokeUint8Typex_Y(key uint8, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Typex_Y) + return ret +} + +func (e *emitNative) invokeETUint8Typex_Y(t typex.EventTime, key uint8, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint8Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint8Typex_Z) + return ret +} + +func (e *emitNative) invokeUint8Typex_Z(key uint8, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint8Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint8Typex_Z) + return ret +} + +func (e *emitNative) invokeETUint8Typex_Z(t typex.EventTime, key uint8, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16) + return ret +} + +func (e *emitNative) invokeUint16(elm uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16) + return ret +} + +func (e *emitNative) invokeETUint16(t typex.EventTime, elm uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16ByteSlice) + return ret +} + +func (e *emitNative) invokeUint16ByteSlice(key uint16, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16ByteSlice) + return ret +} + +func (e *emitNative) invokeETUint16ByteSlice(t typex.EventTime, key uint16, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Bool) + return ret +} + +func (e *emitNative) invokeUint16Bool(key uint16, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Bool) + return ret +} + +func (e *emitNative) invokeETUint16Bool(t typex.EventTime, key uint16, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16String) + return ret +} + +func (e *emitNative) invokeUint16String(key uint16, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16String) + return ret +} + +func (e *emitNative) invokeETUint16String(t typex.EventTime, key uint16, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Int) + return ret +} + +func (e *emitNative) invokeUint16Int(key uint16, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Int) + return ret +} + +func (e *emitNative) invokeETUint16Int(t typex.EventTime, key uint16, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Int8) + return ret +} + +func (e *emitNative) invokeUint16Int8(key uint16, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Int8) + return ret +} + +func (e *emitNative) invokeETUint16Int8(t typex.EventTime, key uint16, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Int16) + return ret +} + +func (e *emitNative) invokeUint16Int16(key uint16, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Int16) + return ret +} + +func (e *emitNative) invokeETUint16Int16(t typex.EventTime, key uint16, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Int32) + return ret +} + +func (e *emitNative) invokeUint16Int32(key uint16, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Int32) + return ret +} + +func (e *emitNative) invokeETUint16Int32(t typex.EventTime, key uint16, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Int64) + return ret +} + +func (e *emitNative) invokeUint16Int64(key uint16, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Int64) + return ret +} + +func (e *emitNative) invokeETUint16Int64(t typex.EventTime, key uint16, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Uint) + return ret +} + +func (e *emitNative) invokeUint16Uint(key uint16, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Uint) + return ret +} + +func (e *emitNative) invokeETUint16Uint(t typex.EventTime, key uint16, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Uint8) + return ret +} + +func (e *emitNative) invokeUint16Uint8(key uint16, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Uint8) + return ret +} + +func (e *emitNative) invokeETUint16Uint8(t typex.EventTime, key uint16, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Uint16) + return ret +} + +func (e *emitNative) invokeUint16Uint16(key uint16, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Uint16) + return ret +} + +func (e *emitNative) invokeETUint16Uint16(t typex.EventTime, key uint16, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Uint32) + return ret +} + +func (e *emitNative) invokeUint16Uint32(key uint16, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Uint32) + return ret +} + +func (e *emitNative) invokeETUint16Uint32(t typex.EventTime, key uint16, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Uint64) + return ret +} + +func (e *emitNative) invokeUint16Uint64(key uint16, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Uint64) + return ret +} + +func (e *emitNative) invokeETUint16Uint64(t typex.EventTime, key uint16, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Float32) + return ret +} + +func (e *emitNative) invokeUint16Float32(key uint16, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Float32) + return ret +} + +func (e *emitNative) invokeETUint16Float32(t typex.EventTime, key uint16, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Float64) + return ret +} + +func (e *emitNative) invokeUint16Float64(key uint16, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Float64) + return ret +} + +func (e *emitNative) invokeETUint16Float64(t typex.EventTime, key uint16, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Typex_T) + return ret +} + +func (e *emitNative) invokeUint16Typex_T(key uint16, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Typex_T) + return ret +} + +func (e *emitNative) invokeETUint16Typex_T(t typex.EventTime, key uint16, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Typex_U) + return ret +} + +func (e *emitNative) invokeUint16Typex_U(key uint16, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Typex_U) + return ret +} + +func (e *emitNative) invokeETUint16Typex_U(t typex.EventTime, key uint16, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Typex_V) + return ret +} + +func (e *emitNative) invokeUint16Typex_V(key uint16, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Typex_V) + return ret +} + +func (e *emitNative) invokeETUint16Typex_V(t typex.EventTime, key uint16, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Typex_W) + return ret +} + +func (e *emitNative) invokeUint16Typex_W(key uint16, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Typex_W) + return ret +} + +func (e *emitNative) invokeETUint16Typex_W(t typex.EventTime, key uint16, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Typex_X) + return ret +} + +func (e *emitNative) invokeUint16Typex_X(key uint16, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Typex_X) + return ret +} + +func (e *emitNative) invokeETUint16Typex_X(t typex.EventTime, key uint16, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Typex_Y) + return ret +} + +func (e *emitNative) invokeUint16Typex_Y(key uint16, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Typex_Y) + return ret +} + +func (e *emitNative) invokeETUint16Typex_Y(t typex.EventTime, key uint16, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint16Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint16Typex_Z) + return ret +} + +func (e *emitNative) invokeUint16Typex_Z(key uint16, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint16Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint16Typex_Z) + return ret +} + +func (e *emitNative) invokeETUint16Typex_Z(t typex.EventTime, key uint16, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32) + return ret +} + +func (e *emitNative) invokeUint32(elm uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32) + return ret +} + +func (e *emitNative) invokeETUint32(t typex.EventTime, elm uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32ByteSlice) + return ret +} + +func (e *emitNative) invokeUint32ByteSlice(key uint32, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32ByteSlice) + return ret +} + +func (e *emitNative) invokeETUint32ByteSlice(t typex.EventTime, key uint32, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Bool) + return ret +} + +func (e *emitNative) invokeUint32Bool(key uint32, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Bool) + return ret +} + +func (e *emitNative) invokeETUint32Bool(t typex.EventTime, key uint32, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32String) + return ret +} + +func (e *emitNative) invokeUint32String(key uint32, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32String) + return ret +} + +func (e *emitNative) invokeETUint32String(t typex.EventTime, key uint32, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Int) + return ret +} + +func (e *emitNative) invokeUint32Int(key uint32, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Int) + return ret +} + +func (e *emitNative) invokeETUint32Int(t typex.EventTime, key uint32, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Int8) + return ret +} + +func (e *emitNative) invokeUint32Int8(key uint32, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Int8) + return ret +} + +func (e *emitNative) invokeETUint32Int8(t typex.EventTime, key uint32, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Int16) + return ret +} + +func (e *emitNative) invokeUint32Int16(key uint32, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Int16) + return ret +} + +func (e *emitNative) invokeETUint32Int16(t typex.EventTime, key uint32, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Int32) + return ret +} + +func (e *emitNative) invokeUint32Int32(key uint32, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Int32) + return ret +} + +func (e *emitNative) invokeETUint32Int32(t typex.EventTime, key uint32, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Int64) + return ret +} + +func (e *emitNative) invokeUint32Int64(key uint32, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Int64) + return ret +} + +func (e *emitNative) invokeETUint32Int64(t typex.EventTime, key uint32, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Uint) + return ret +} + +func (e *emitNative) invokeUint32Uint(key uint32, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Uint) + return ret +} + +func (e *emitNative) invokeETUint32Uint(t typex.EventTime, key uint32, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Uint8) + return ret +} + +func (e *emitNative) invokeUint32Uint8(key uint32, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Uint8) + return ret +} + +func (e *emitNative) invokeETUint32Uint8(t typex.EventTime, key uint32, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Uint16) + return ret +} + +func (e *emitNative) invokeUint32Uint16(key uint32, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Uint16) + return ret +} + +func (e *emitNative) invokeETUint32Uint16(t typex.EventTime, key uint32, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Uint32) + return ret +} + +func (e *emitNative) invokeUint32Uint32(key uint32, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Uint32) + return ret +} + +func (e *emitNative) invokeETUint32Uint32(t typex.EventTime, key uint32, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Uint64) + return ret +} + +func (e *emitNative) invokeUint32Uint64(key uint32, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Uint64) + return ret +} + +func (e *emitNative) invokeETUint32Uint64(t typex.EventTime, key uint32, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Float32) + return ret +} + +func (e *emitNative) invokeUint32Float32(key uint32, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Float32) + return ret +} + +func (e *emitNative) invokeETUint32Float32(t typex.EventTime, key uint32, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Float64) + return ret +} + +func (e *emitNative) invokeUint32Float64(key uint32, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Float64) + return ret +} + +func (e *emitNative) invokeETUint32Float64(t typex.EventTime, key uint32, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Typex_T) + return ret +} + +func (e *emitNative) invokeUint32Typex_T(key uint32, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Typex_T) + return ret +} + +func (e *emitNative) invokeETUint32Typex_T(t typex.EventTime, key uint32, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Typex_U) + return ret +} + +func (e *emitNative) invokeUint32Typex_U(key uint32, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Typex_U) + return ret +} + +func (e *emitNative) invokeETUint32Typex_U(t typex.EventTime, key uint32, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Typex_V) + return ret +} + +func (e *emitNative) invokeUint32Typex_V(key uint32, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Typex_V) + return ret +} + +func (e *emitNative) invokeETUint32Typex_V(t typex.EventTime, key uint32, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Typex_W) + return ret +} + +func (e *emitNative) invokeUint32Typex_W(key uint32, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Typex_W) + return ret +} + +func (e *emitNative) invokeETUint32Typex_W(t typex.EventTime, key uint32, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Typex_X) + return ret +} + +func (e *emitNative) invokeUint32Typex_X(key uint32, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Typex_X) + return ret +} + +func (e *emitNative) invokeETUint32Typex_X(t typex.EventTime, key uint32, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Typex_Y) + return ret +} + +func (e *emitNative) invokeUint32Typex_Y(key uint32, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Typex_Y) + return ret +} + +func (e *emitNative) invokeETUint32Typex_Y(t typex.EventTime, key uint32, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint32Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint32Typex_Z) + return ret +} + +func (e *emitNative) invokeUint32Typex_Z(key uint32, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint32Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint32Typex_Z) + return ret +} + +func (e *emitNative) invokeETUint32Typex_Z(t typex.EventTime, key uint32, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64) + return ret +} + +func (e *emitNative) invokeUint64(elm uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64) + return ret +} + +func (e *emitNative) invokeETUint64(t typex.EventTime, elm uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64ByteSlice) + return ret +} + +func (e *emitNative) invokeUint64ByteSlice(key uint64, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64ByteSlice) + return ret +} + +func (e *emitNative) invokeETUint64ByteSlice(t typex.EventTime, key uint64, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Bool) + return ret +} + +func (e *emitNative) invokeUint64Bool(key uint64, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Bool) + return ret +} + +func (e *emitNative) invokeETUint64Bool(t typex.EventTime, key uint64, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64String) + return ret +} + +func (e *emitNative) invokeUint64String(key uint64, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64String) + return ret +} + +func (e *emitNative) invokeETUint64String(t typex.EventTime, key uint64, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Int) + return ret +} + +func (e *emitNative) invokeUint64Int(key uint64, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Int) + return ret +} + +func (e *emitNative) invokeETUint64Int(t typex.EventTime, key uint64, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Int8) + return ret +} + +func (e *emitNative) invokeUint64Int8(key uint64, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Int8) + return ret +} + +func (e *emitNative) invokeETUint64Int8(t typex.EventTime, key uint64, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Int16) + return ret +} + +func (e *emitNative) invokeUint64Int16(key uint64, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Int16) + return ret +} + +func (e *emitNative) invokeETUint64Int16(t typex.EventTime, key uint64, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Int32) + return ret +} + +func (e *emitNative) invokeUint64Int32(key uint64, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Int32) + return ret +} + +func (e *emitNative) invokeETUint64Int32(t typex.EventTime, key uint64, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Int64) + return ret +} + +func (e *emitNative) invokeUint64Int64(key uint64, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Int64) + return ret +} + +func (e *emitNative) invokeETUint64Int64(t typex.EventTime, key uint64, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Uint) + return ret +} + +func (e *emitNative) invokeUint64Uint(key uint64, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Uint) + return ret +} + +func (e *emitNative) invokeETUint64Uint(t typex.EventTime, key uint64, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Uint8) + return ret +} + +func (e *emitNative) invokeUint64Uint8(key uint64, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Uint8) + return ret +} + +func (e *emitNative) invokeETUint64Uint8(t typex.EventTime, key uint64, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Uint16) + return ret +} + +func (e *emitNative) invokeUint64Uint16(key uint64, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Uint16) + return ret +} + +func (e *emitNative) invokeETUint64Uint16(t typex.EventTime, key uint64, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Uint32) + return ret +} + +func (e *emitNative) invokeUint64Uint32(key uint64, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Uint32) + return ret +} + +func (e *emitNative) invokeETUint64Uint32(t typex.EventTime, key uint64, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Uint64) + return ret +} + +func (e *emitNative) invokeUint64Uint64(key uint64, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Uint64) + return ret +} + +func (e *emitNative) invokeETUint64Uint64(t typex.EventTime, key uint64, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Float32) + return ret +} + +func (e *emitNative) invokeUint64Float32(key uint64, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Float32) + return ret +} + +func (e *emitNative) invokeETUint64Float32(t typex.EventTime, key uint64, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Float64) + return ret +} + +func (e *emitNative) invokeUint64Float64(key uint64, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Float64) + return ret +} + +func (e *emitNative) invokeETUint64Float64(t typex.EventTime, key uint64, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Typex_T) + return ret +} + +func (e *emitNative) invokeUint64Typex_T(key uint64, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Typex_T) + return ret +} + +func (e *emitNative) invokeETUint64Typex_T(t typex.EventTime, key uint64, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Typex_U) + return ret +} + +func (e *emitNative) invokeUint64Typex_U(key uint64, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Typex_U) + return ret +} + +func (e *emitNative) invokeETUint64Typex_U(t typex.EventTime, key uint64, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Typex_V) + return ret +} + +func (e *emitNative) invokeUint64Typex_V(key uint64, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Typex_V) + return ret +} + +func (e *emitNative) invokeETUint64Typex_V(t typex.EventTime, key uint64, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Typex_W) + return ret +} + +func (e *emitNative) invokeUint64Typex_W(key uint64, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Typex_W) + return ret +} + +func (e *emitNative) invokeETUint64Typex_W(t typex.EventTime, key uint64, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Typex_X) + return ret +} + +func (e *emitNative) invokeUint64Typex_X(key uint64, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Typex_X) + return ret +} + +func (e *emitNative) invokeETUint64Typex_X(t typex.EventTime, key uint64, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Typex_Y) + return ret +} + +func (e *emitNative) invokeUint64Typex_Y(key uint64, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Typex_Y) + return ret +} + +func (e *emitNative) invokeETUint64Typex_Y(t typex.EventTime, key uint64, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerUint64Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeUint64Typex_Z) + return ret +} + +func (e *emitNative) invokeUint64Typex_Z(key uint64, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETUint64Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETUint64Typex_Z) + return ret +} + +func (e *emitNative) invokeETUint64Typex_Z(t typex.EventTime, key uint64, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32) + return ret +} + +func (e *emitNative) invokeFloat32(elm float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32) + return ret +} + +func (e *emitNative) invokeETFloat32(t typex.EventTime, elm float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32ByteSlice) + return ret +} + +func (e *emitNative) invokeFloat32ByteSlice(key float32, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32ByteSlice) + return ret +} + +func (e *emitNative) invokeETFloat32ByteSlice(t typex.EventTime, key float32, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Bool) + return ret +} + +func (e *emitNative) invokeFloat32Bool(key float32, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Bool) + return ret +} + +func (e *emitNative) invokeETFloat32Bool(t typex.EventTime, key float32, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32String) + return ret +} + +func (e *emitNative) invokeFloat32String(key float32, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32String) + return ret +} + +func (e *emitNative) invokeETFloat32String(t typex.EventTime, key float32, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Int) + return ret +} + +func (e *emitNative) invokeFloat32Int(key float32, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Int) + return ret +} + +func (e *emitNative) invokeETFloat32Int(t typex.EventTime, key float32, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Int8) + return ret +} + +func (e *emitNative) invokeFloat32Int8(key float32, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Int8) + return ret +} + +func (e *emitNative) invokeETFloat32Int8(t typex.EventTime, key float32, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Int16) + return ret +} + +func (e *emitNative) invokeFloat32Int16(key float32, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Int16) + return ret +} + +func (e *emitNative) invokeETFloat32Int16(t typex.EventTime, key float32, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Int32) + return ret +} + +func (e *emitNative) invokeFloat32Int32(key float32, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Int32) + return ret +} + +func (e *emitNative) invokeETFloat32Int32(t typex.EventTime, key float32, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Int64) + return ret +} + +func (e *emitNative) invokeFloat32Int64(key float32, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Int64) + return ret +} + +func (e *emitNative) invokeETFloat32Int64(t typex.EventTime, key float32, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Uint) + return ret +} + +func (e *emitNative) invokeFloat32Uint(key float32, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Uint) + return ret +} + +func (e *emitNative) invokeETFloat32Uint(t typex.EventTime, key float32, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Uint8) + return ret +} + +func (e *emitNative) invokeFloat32Uint8(key float32, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Uint8) + return ret +} + +func (e *emitNative) invokeETFloat32Uint8(t typex.EventTime, key float32, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Uint16) + return ret +} + +func (e *emitNative) invokeFloat32Uint16(key float32, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Uint16) + return ret +} + +func (e *emitNative) invokeETFloat32Uint16(t typex.EventTime, key float32, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Uint32) + return ret +} + +func (e *emitNative) invokeFloat32Uint32(key float32, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Uint32) + return ret +} + +func (e *emitNative) invokeETFloat32Uint32(t typex.EventTime, key float32, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Uint64) + return ret +} + +func (e *emitNative) invokeFloat32Uint64(key float32, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Uint64) + return ret +} + +func (e *emitNative) invokeETFloat32Uint64(t typex.EventTime, key float32, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Float32) + return ret +} + +func (e *emitNative) invokeFloat32Float32(key float32, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Float32) + return ret +} + +func (e *emitNative) invokeETFloat32Float32(t typex.EventTime, key float32, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Float64) + return ret +} + +func (e *emitNative) invokeFloat32Float64(key float32, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Float64) + return ret +} + +func (e *emitNative) invokeETFloat32Float64(t typex.EventTime, key float32, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Typex_T) + return ret +} + +func (e *emitNative) invokeFloat32Typex_T(key float32, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Typex_T) + return ret +} + +func (e *emitNative) invokeETFloat32Typex_T(t typex.EventTime, key float32, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Typex_U) + return ret +} + +func (e *emitNative) invokeFloat32Typex_U(key float32, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Typex_U) + return ret +} + +func (e *emitNative) invokeETFloat32Typex_U(t typex.EventTime, key float32, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Typex_V) + return ret +} + +func (e *emitNative) invokeFloat32Typex_V(key float32, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Typex_V) + return ret +} + +func (e *emitNative) invokeETFloat32Typex_V(t typex.EventTime, key float32, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Typex_W) + return ret +} + +func (e *emitNative) invokeFloat32Typex_W(key float32, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Typex_W) + return ret +} + +func (e *emitNative) invokeETFloat32Typex_W(t typex.EventTime, key float32, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Typex_X) + return ret +} + +func (e *emitNative) invokeFloat32Typex_X(key float32, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Typex_X) + return ret +} + +func (e *emitNative) invokeETFloat32Typex_X(t typex.EventTime, key float32, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Typex_Y) + return ret +} + +func (e *emitNative) invokeFloat32Typex_Y(key float32, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Typex_Y) + return ret +} + +func (e *emitNative) invokeETFloat32Typex_Y(t typex.EventTime, key float32, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat32Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat32Typex_Z) + return ret +} + +func (e *emitNative) invokeFloat32Typex_Z(key float32, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat32Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat32Typex_Z) + return ret +} + +func (e *emitNative) invokeETFloat32Typex_Z(t typex.EventTime, key float32, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64) + return ret +} + +func (e *emitNative) invokeFloat64(elm float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64) + return ret +} + +func (e *emitNative) invokeETFloat64(t typex.EventTime, elm float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64ByteSlice) + return ret +} + +func (e *emitNative) invokeFloat64ByteSlice(key float64, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64ByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64ByteSlice) + return ret +} + +func (e *emitNative) invokeETFloat64ByteSlice(t typex.EventTime, key float64, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Bool) + return ret +} + +func (e *emitNative) invokeFloat64Bool(key float64, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Bool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Bool) + return ret +} + +func (e *emitNative) invokeETFloat64Bool(t typex.EventTime, key float64, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64String) + return ret +} + +func (e *emitNative) invokeFloat64String(key float64, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64String(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64String) + return ret +} + +func (e *emitNative) invokeETFloat64String(t typex.EventTime, key float64, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Int) + return ret +} + +func (e *emitNative) invokeFloat64Int(key float64, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Int(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Int) + return ret +} + +func (e *emitNative) invokeETFloat64Int(t typex.EventTime, key float64, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Int8) + return ret +} + +func (e *emitNative) invokeFloat64Int8(key float64, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Int8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Int8) + return ret +} + +func (e *emitNative) invokeETFloat64Int8(t typex.EventTime, key float64, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Int16) + return ret +} + +func (e *emitNative) invokeFloat64Int16(key float64, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Int16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Int16) + return ret +} + +func (e *emitNative) invokeETFloat64Int16(t typex.EventTime, key float64, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Int32) + return ret +} + +func (e *emitNative) invokeFloat64Int32(key float64, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Int32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Int32) + return ret +} + +func (e *emitNative) invokeETFloat64Int32(t typex.EventTime, key float64, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Int64) + return ret +} + +func (e *emitNative) invokeFloat64Int64(key float64, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Int64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Int64) + return ret +} + +func (e *emitNative) invokeETFloat64Int64(t typex.EventTime, key float64, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Uint) + return ret +} + +func (e *emitNative) invokeFloat64Uint(key float64, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Uint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Uint) + return ret +} + +func (e *emitNative) invokeETFloat64Uint(t typex.EventTime, key float64, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Uint8) + return ret +} + +func (e *emitNative) invokeFloat64Uint8(key float64, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Uint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Uint8) + return ret +} + +func (e *emitNative) invokeETFloat64Uint8(t typex.EventTime, key float64, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Uint16) + return ret +} + +func (e *emitNative) invokeFloat64Uint16(key float64, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Uint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Uint16) + return ret +} + +func (e *emitNative) invokeETFloat64Uint16(t typex.EventTime, key float64, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Uint32) + return ret +} + +func (e *emitNative) invokeFloat64Uint32(key float64, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Uint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Uint32) + return ret +} + +func (e *emitNative) invokeETFloat64Uint32(t typex.EventTime, key float64, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Uint64) + return ret +} + +func (e *emitNative) invokeFloat64Uint64(key float64, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Uint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Uint64) + return ret +} + +func (e *emitNative) invokeETFloat64Uint64(t typex.EventTime, key float64, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Float32) + return ret +} + +func (e *emitNative) invokeFloat64Float32(key float64, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Float32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Float32) + return ret +} + +func (e *emitNative) invokeETFloat64Float32(t typex.EventTime, key float64, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Float64) + return ret +} + +func (e *emitNative) invokeFloat64Float64(key float64, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Float64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Float64) + return ret +} + +func (e *emitNative) invokeETFloat64Float64(t typex.EventTime, key float64, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Typex_T) + return ret +} + +func (e *emitNative) invokeFloat64Typex_T(key float64, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Typex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Typex_T) + return ret +} + +func (e *emitNative) invokeETFloat64Typex_T(t typex.EventTime, key float64, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Typex_U) + return ret +} + +func (e *emitNative) invokeFloat64Typex_U(key float64, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Typex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Typex_U) + return ret +} + +func (e *emitNative) invokeETFloat64Typex_U(t typex.EventTime, key float64, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Typex_V) + return ret +} + +func (e *emitNative) invokeFloat64Typex_V(key float64, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Typex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Typex_V) + return ret +} + +func (e *emitNative) invokeETFloat64Typex_V(t typex.EventTime, key float64, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Typex_W) + return ret +} + +func (e *emitNative) invokeFloat64Typex_W(key float64, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Typex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Typex_W) + return ret +} + +func (e *emitNative) invokeETFloat64Typex_W(t typex.EventTime, key float64, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Typex_X) + return ret +} + +func (e *emitNative) invokeFloat64Typex_X(key float64, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Typex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Typex_X) + return ret +} + +func (e *emitNative) invokeETFloat64Typex_X(t typex.EventTime, key float64, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Typex_Y) + return ret +} + +func (e *emitNative) invokeFloat64Typex_Y(key float64, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Typex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Typex_Y) + return ret +} + +func (e *emitNative) invokeETFloat64Typex_Y(t typex.EventTime, key float64, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerFloat64Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeFloat64Typex_Z) + return ret +} + +func (e *emitNative) invokeFloat64Typex_Z(key float64, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETFloat64Typex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETFloat64Typex_Z) + return ret +} + +func (e *emitNative) invokeETFloat64Typex_Z(t typex.EventTime, key float64, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_T) + return ret +} + +func (e *emitNative) invokeTypex_T(elm typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_T) + return ret +} + +func (e *emitNative) invokeETTypex_T(t typex.EventTime, elm typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TByteSlice) + return ret +} + +func (e *emitNative) invokeTypex_TByteSlice(key typex.T, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TByteSlice) + return ret +} + +func (e *emitNative) invokeETTypex_TByteSlice(t typex.EventTime, key typex.T, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TBool) + return ret +} + +func (e *emitNative) invokeTypex_TBool(key typex.T, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TBool) + return ret +} + +func (e *emitNative) invokeETTypex_TBool(t typex.EventTime, key typex.T, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TString) + return ret +} + +func (e *emitNative) invokeTypex_TString(key typex.T, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TString) + return ret +} + +func (e *emitNative) invokeETTypex_TString(t typex.EventTime, key typex.T, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TInt) + return ret +} + +func (e *emitNative) invokeTypex_TInt(key typex.T, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TInt) + return ret +} + +func (e *emitNative) invokeETTypex_TInt(t typex.EventTime, key typex.T, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TInt8) + return ret +} + +func (e *emitNative) invokeTypex_TInt8(key typex.T, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TInt8) + return ret +} + +func (e *emitNative) invokeETTypex_TInt8(t typex.EventTime, key typex.T, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TInt16) + return ret +} + +func (e *emitNative) invokeTypex_TInt16(key typex.T, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TInt16) + return ret +} + +func (e *emitNative) invokeETTypex_TInt16(t typex.EventTime, key typex.T, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TInt32) + return ret +} + +func (e *emitNative) invokeTypex_TInt32(key typex.T, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TInt32) + return ret +} + +func (e *emitNative) invokeETTypex_TInt32(t typex.EventTime, key typex.T, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TInt64) + return ret +} + +func (e *emitNative) invokeTypex_TInt64(key typex.T, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TInt64) + return ret +} + +func (e *emitNative) invokeETTypex_TInt64(t typex.EventTime, key typex.T, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TUint) + return ret +} + +func (e *emitNative) invokeTypex_TUint(key typex.T, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TUint) + return ret +} + +func (e *emitNative) invokeETTypex_TUint(t typex.EventTime, key typex.T, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TUint8) + return ret +} + +func (e *emitNative) invokeTypex_TUint8(key typex.T, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TUint8) + return ret +} + +func (e *emitNative) invokeETTypex_TUint8(t typex.EventTime, key typex.T, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TUint16) + return ret +} + +func (e *emitNative) invokeTypex_TUint16(key typex.T, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TUint16) + return ret +} + +func (e *emitNative) invokeETTypex_TUint16(t typex.EventTime, key typex.T, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TUint32) + return ret +} + +func (e *emitNative) invokeTypex_TUint32(key typex.T, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TUint32) + return ret +} + +func (e *emitNative) invokeETTypex_TUint32(t typex.EventTime, key typex.T, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TUint64) + return ret +} + +func (e *emitNative) invokeTypex_TUint64(key typex.T, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TUint64) + return ret +} + +func (e *emitNative) invokeETTypex_TUint64(t typex.EventTime, key typex.T, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TFloat32) + return ret +} + +func (e *emitNative) invokeTypex_TFloat32(key typex.T, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TFloat32) + return ret +} + +func (e *emitNative) invokeETTypex_TFloat32(t typex.EventTime, key typex.T, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TFloat64) + return ret +} + +func (e *emitNative) invokeTypex_TFloat64(key typex.T, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TFloat64) + return ret +} + +func (e *emitNative) invokeETTypex_TFloat64(t typex.EventTime, key typex.T, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TTypex_T) + return ret +} + +func (e *emitNative) invokeTypex_TTypex_T(key typex.T, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TTypex_T) + return ret +} + +func (e *emitNative) invokeETTypex_TTypex_T(t typex.EventTime, key typex.T, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TTypex_U) + return ret +} + +func (e *emitNative) invokeTypex_TTypex_U(key typex.T, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TTypex_U) + return ret +} + +func (e *emitNative) invokeETTypex_TTypex_U(t typex.EventTime, key typex.T, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TTypex_V) + return ret +} + +func (e *emitNative) invokeTypex_TTypex_V(key typex.T, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TTypex_V) + return ret +} + +func (e *emitNative) invokeETTypex_TTypex_V(t typex.EventTime, key typex.T, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TTypex_W) + return ret +} + +func (e *emitNative) invokeTypex_TTypex_W(key typex.T, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TTypex_W) + return ret +} + +func (e *emitNative) invokeETTypex_TTypex_W(t typex.EventTime, key typex.T, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TTypex_X) + return ret +} + +func (e *emitNative) invokeTypex_TTypex_X(key typex.T, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TTypex_X) + return ret +} + +func (e *emitNative) invokeETTypex_TTypex_X(t typex.EventTime, key typex.T, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TTypex_Y) + return ret +} + +func (e *emitNative) invokeTypex_TTypex_Y(key typex.T, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TTypex_Y) + return ret +} + +func (e *emitNative) invokeETTypex_TTypex_Y(t typex.EventTime, key typex.T, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_TTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_TTypex_Z) + return ret +} + +func (e *emitNative) invokeTypex_TTypex_Z(key typex.T, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_TTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_TTypex_Z) + return ret +} + +func (e *emitNative) invokeETTypex_TTypex_Z(t typex.EventTime, key typex.T, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_U) + return ret +} + +func (e *emitNative) invokeTypex_U(elm typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_U) + return ret +} + +func (e *emitNative) invokeETTypex_U(t typex.EventTime, elm typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UByteSlice) + return ret +} + +func (e *emitNative) invokeTypex_UByteSlice(key typex.U, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UByteSlice) + return ret +} + +func (e *emitNative) invokeETTypex_UByteSlice(t typex.EventTime, key typex.U, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UBool) + return ret +} + +func (e *emitNative) invokeTypex_UBool(key typex.U, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UBool) + return ret +} + +func (e *emitNative) invokeETTypex_UBool(t typex.EventTime, key typex.U, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UString) + return ret +} + +func (e *emitNative) invokeTypex_UString(key typex.U, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UString) + return ret +} + +func (e *emitNative) invokeETTypex_UString(t typex.EventTime, key typex.U, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UInt) + return ret +} + +func (e *emitNative) invokeTypex_UInt(key typex.U, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UInt) + return ret +} + +func (e *emitNative) invokeETTypex_UInt(t typex.EventTime, key typex.U, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UInt8) + return ret +} + +func (e *emitNative) invokeTypex_UInt8(key typex.U, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UInt8) + return ret +} + +func (e *emitNative) invokeETTypex_UInt8(t typex.EventTime, key typex.U, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UInt16) + return ret +} + +func (e *emitNative) invokeTypex_UInt16(key typex.U, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UInt16) + return ret +} + +func (e *emitNative) invokeETTypex_UInt16(t typex.EventTime, key typex.U, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UInt32) + return ret +} + +func (e *emitNative) invokeTypex_UInt32(key typex.U, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UInt32) + return ret +} + +func (e *emitNative) invokeETTypex_UInt32(t typex.EventTime, key typex.U, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UInt64) + return ret +} + +func (e *emitNative) invokeTypex_UInt64(key typex.U, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UInt64) + return ret +} + +func (e *emitNative) invokeETTypex_UInt64(t typex.EventTime, key typex.U, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UUint) + return ret +} + +func (e *emitNative) invokeTypex_UUint(key typex.U, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UUint) + return ret +} + +func (e *emitNative) invokeETTypex_UUint(t typex.EventTime, key typex.U, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UUint8) + return ret +} + +func (e *emitNative) invokeTypex_UUint8(key typex.U, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UUint8) + return ret +} + +func (e *emitNative) invokeETTypex_UUint8(t typex.EventTime, key typex.U, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UUint16) + return ret +} + +func (e *emitNative) invokeTypex_UUint16(key typex.U, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UUint16) + return ret +} + +func (e *emitNative) invokeETTypex_UUint16(t typex.EventTime, key typex.U, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UUint32) + return ret +} + +func (e *emitNative) invokeTypex_UUint32(key typex.U, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UUint32) + return ret +} + +func (e *emitNative) invokeETTypex_UUint32(t typex.EventTime, key typex.U, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UUint64) + return ret +} + +func (e *emitNative) invokeTypex_UUint64(key typex.U, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UUint64) + return ret +} + +func (e *emitNative) invokeETTypex_UUint64(t typex.EventTime, key typex.U, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UFloat32) + return ret +} + +func (e *emitNative) invokeTypex_UFloat32(key typex.U, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UFloat32) + return ret +} + +func (e *emitNative) invokeETTypex_UFloat32(t typex.EventTime, key typex.U, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UFloat64) + return ret +} + +func (e *emitNative) invokeTypex_UFloat64(key typex.U, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UFloat64) + return ret +} + +func (e *emitNative) invokeETTypex_UFloat64(t typex.EventTime, key typex.U, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UTypex_T) + return ret +} + +func (e *emitNative) invokeTypex_UTypex_T(key typex.U, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UTypex_T) + return ret +} + +func (e *emitNative) invokeETTypex_UTypex_T(t typex.EventTime, key typex.U, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UTypex_U) + return ret +} + +func (e *emitNative) invokeTypex_UTypex_U(key typex.U, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UTypex_U) + return ret +} + +func (e *emitNative) invokeETTypex_UTypex_U(t typex.EventTime, key typex.U, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UTypex_V) + return ret +} + +func (e *emitNative) invokeTypex_UTypex_V(key typex.U, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UTypex_V) + return ret +} + +func (e *emitNative) invokeETTypex_UTypex_V(t typex.EventTime, key typex.U, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UTypex_W) + return ret +} + +func (e *emitNative) invokeTypex_UTypex_W(key typex.U, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UTypex_W) + return ret +} + +func (e *emitNative) invokeETTypex_UTypex_W(t typex.EventTime, key typex.U, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UTypex_X) + return ret +} + +func (e *emitNative) invokeTypex_UTypex_X(key typex.U, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UTypex_X) + return ret +} + +func (e *emitNative) invokeETTypex_UTypex_X(t typex.EventTime, key typex.U, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UTypex_Y) + return ret +} + +func (e *emitNative) invokeTypex_UTypex_Y(key typex.U, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UTypex_Y) + return ret +} + +func (e *emitNative) invokeETTypex_UTypex_Y(t typex.EventTime, key typex.U, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_UTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_UTypex_Z) + return ret +} + +func (e *emitNative) invokeTypex_UTypex_Z(key typex.U, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_UTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_UTypex_Z) + return ret +} + +func (e *emitNative) invokeETTypex_UTypex_Z(t typex.EventTime, key typex.U, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_V) + return ret +} + +func (e *emitNative) invokeTypex_V(elm typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_V) + return ret +} + +func (e *emitNative) invokeETTypex_V(t typex.EventTime, elm typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VByteSlice) + return ret +} + +func (e *emitNative) invokeTypex_VByteSlice(key typex.V, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VByteSlice) + return ret +} + +func (e *emitNative) invokeETTypex_VByteSlice(t typex.EventTime, key typex.V, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VBool) + return ret +} + +func (e *emitNative) invokeTypex_VBool(key typex.V, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VBool) + return ret +} + +func (e *emitNative) invokeETTypex_VBool(t typex.EventTime, key typex.V, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VString) + return ret +} + +func (e *emitNative) invokeTypex_VString(key typex.V, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VString) + return ret +} + +func (e *emitNative) invokeETTypex_VString(t typex.EventTime, key typex.V, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VInt) + return ret +} + +func (e *emitNative) invokeTypex_VInt(key typex.V, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VInt) + return ret +} + +func (e *emitNative) invokeETTypex_VInt(t typex.EventTime, key typex.V, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VInt8) + return ret +} + +func (e *emitNative) invokeTypex_VInt8(key typex.V, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VInt8) + return ret +} + +func (e *emitNative) invokeETTypex_VInt8(t typex.EventTime, key typex.V, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VInt16) + return ret +} + +func (e *emitNative) invokeTypex_VInt16(key typex.V, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VInt16) + return ret +} + +func (e *emitNative) invokeETTypex_VInt16(t typex.EventTime, key typex.V, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VInt32) + return ret +} + +func (e *emitNative) invokeTypex_VInt32(key typex.V, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VInt32) + return ret +} + +func (e *emitNative) invokeETTypex_VInt32(t typex.EventTime, key typex.V, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VInt64) + return ret +} + +func (e *emitNative) invokeTypex_VInt64(key typex.V, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VInt64) + return ret +} + +func (e *emitNative) invokeETTypex_VInt64(t typex.EventTime, key typex.V, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VUint) + return ret +} + +func (e *emitNative) invokeTypex_VUint(key typex.V, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VUint) + return ret +} + +func (e *emitNative) invokeETTypex_VUint(t typex.EventTime, key typex.V, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VUint8) + return ret +} + +func (e *emitNative) invokeTypex_VUint8(key typex.V, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VUint8) + return ret +} + +func (e *emitNative) invokeETTypex_VUint8(t typex.EventTime, key typex.V, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VUint16) + return ret +} + +func (e *emitNative) invokeTypex_VUint16(key typex.V, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VUint16) + return ret +} + +func (e *emitNative) invokeETTypex_VUint16(t typex.EventTime, key typex.V, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VUint32) + return ret +} + +func (e *emitNative) invokeTypex_VUint32(key typex.V, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VUint32) + return ret +} + +func (e *emitNative) invokeETTypex_VUint32(t typex.EventTime, key typex.V, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VUint64) + return ret +} + +func (e *emitNative) invokeTypex_VUint64(key typex.V, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VUint64) + return ret +} + +func (e *emitNative) invokeETTypex_VUint64(t typex.EventTime, key typex.V, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VFloat32) + return ret +} + +func (e *emitNative) invokeTypex_VFloat32(key typex.V, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VFloat32) + return ret +} + +func (e *emitNative) invokeETTypex_VFloat32(t typex.EventTime, key typex.V, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VFloat64) + return ret +} + +func (e *emitNative) invokeTypex_VFloat64(key typex.V, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VFloat64) + return ret +} + +func (e *emitNative) invokeETTypex_VFloat64(t typex.EventTime, key typex.V, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VTypex_T) + return ret +} + +func (e *emitNative) invokeTypex_VTypex_T(key typex.V, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VTypex_T) + return ret +} + +func (e *emitNative) invokeETTypex_VTypex_T(t typex.EventTime, key typex.V, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VTypex_U) + return ret +} + +func (e *emitNative) invokeTypex_VTypex_U(key typex.V, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VTypex_U) + return ret +} + +func (e *emitNative) invokeETTypex_VTypex_U(t typex.EventTime, key typex.V, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VTypex_V) + return ret +} + +func (e *emitNative) invokeTypex_VTypex_V(key typex.V, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VTypex_V) + return ret +} + +func (e *emitNative) invokeETTypex_VTypex_V(t typex.EventTime, key typex.V, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VTypex_W) + return ret +} + +func (e *emitNative) invokeTypex_VTypex_W(key typex.V, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VTypex_W) + return ret +} + +func (e *emitNative) invokeETTypex_VTypex_W(t typex.EventTime, key typex.V, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VTypex_X) + return ret +} + +func (e *emitNative) invokeTypex_VTypex_X(key typex.V, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VTypex_X) + return ret +} + +func (e *emitNative) invokeETTypex_VTypex_X(t typex.EventTime, key typex.V, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VTypex_Y) + return ret +} + +func (e *emitNative) invokeTypex_VTypex_Y(key typex.V, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VTypex_Y) + return ret +} + +func (e *emitNative) invokeETTypex_VTypex_Y(t typex.EventTime, key typex.V, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_VTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_VTypex_Z) + return ret +} + +func (e *emitNative) invokeTypex_VTypex_Z(key typex.V, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_VTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_VTypex_Z) + return ret +} + +func (e *emitNative) invokeETTypex_VTypex_Z(t typex.EventTime, key typex.V, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_W) + return ret +} + +func (e *emitNative) invokeTypex_W(elm typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_W) + return ret +} + +func (e *emitNative) invokeETTypex_W(t typex.EventTime, elm typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WByteSlice) + return ret +} + +func (e *emitNative) invokeTypex_WByteSlice(key typex.W, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WByteSlice) + return ret +} + +func (e *emitNative) invokeETTypex_WByteSlice(t typex.EventTime, key typex.W, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WBool) + return ret +} + +func (e *emitNative) invokeTypex_WBool(key typex.W, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WBool) + return ret +} + +func (e *emitNative) invokeETTypex_WBool(t typex.EventTime, key typex.W, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WString) + return ret +} + +func (e *emitNative) invokeTypex_WString(key typex.W, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WString) + return ret +} + +func (e *emitNative) invokeETTypex_WString(t typex.EventTime, key typex.W, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WInt) + return ret +} + +func (e *emitNative) invokeTypex_WInt(key typex.W, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WInt) + return ret +} + +func (e *emitNative) invokeETTypex_WInt(t typex.EventTime, key typex.W, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WInt8) + return ret +} + +func (e *emitNative) invokeTypex_WInt8(key typex.W, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WInt8) + return ret +} + +func (e *emitNative) invokeETTypex_WInt8(t typex.EventTime, key typex.W, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WInt16) + return ret +} + +func (e *emitNative) invokeTypex_WInt16(key typex.W, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WInt16) + return ret +} + +func (e *emitNative) invokeETTypex_WInt16(t typex.EventTime, key typex.W, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WInt32) + return ret +} + +func (e *emitNative) invokeTypex_WInt32(key typex.W, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WInt32) + return ret +} + +func (e *emitNative) invokeETTypex_WInt32(t typex.EventTime, key typex.W, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WInt64) + return ret +} + +func (e *emitNative) invokeTypex_WInt64(key typex.W, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WInt64) + return ret +} + +func (e *emitNative) invokeETTypex_WInt64(t typex.EventTime, key typex.W, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WUint) + return ret +} + +func (e *emitNative) invokeTypex_WUint(key typex.W, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WUint) + return ret +} + +func (e *emitNative) invokeETTypex_WUint(t typex.EventTime, key typex.W, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WUint8) + return ret +} + +func (e *emitNative) invokeTypex_WUint8(key typex.W, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WUint8) + return ret +} + +func (e *emitNative) invokeETTypex_WUint8(t typex.EventTime, key typex.W, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WUint16) + return ret +} + +func (e *emitNative) invokeTypex_WUint16(key typex.W, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WUint16) + return ret +} + +func (e *emitNative) invokeETTypex_WUint16(t typex.EventTime, key typex.W, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WUint32) + return ret +} + +func (e *emitNative) invokeTypex_WUint32(key typex.W, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WUint32) + return ret +} + +func (e *emitNative) invokeETTypex_WUint32(t typex.EventTime, key typex.W, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WUint64) + return ret +} + +func (e *emitNative) invokeTypex_WUint64(key typex.W, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WUint64) + return ret +} + +func (e *emitNative) invokeETTypex_WUint64(t typex.EventTime, key typex.W, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WFloat32) + return ret +} + +func (e *emitNative) invokeTypex_WFloat32(key typex.W, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WFloat32) + return ret +} + +func (e *emitNative) invokeETTypex_WFloat32(t typex.EventTime, key typex.W, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WFloat64) + return ret +} + +func (e *emitNative) invokeTypex_WFloat64(key typex.W, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WFloat64) + return ret +} + +func (e *emitNative) invokeETTypex_WFloat64(t typex.EventTime, key typex.W, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WTypex_T) + return ret +} + +func (e *emitNative) invokeTypex_WTypex_T(key typex.W, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WTypex_T) + return ret +} + +func (e *emitNative) invokeETTypex_WTypex_T(t typex.EventTime, key typex.W, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WTypex_U) + return ret +} + +func (e *emitNative) invokeTypex_WTypex_U(key typex.W, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WTypex_U) + return ret +} + +func (e *emitNative) invokeETTypex_WTypex_U(t typex.EventTime, key typex.W, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WTypex_V) + return ret +} + +func (e *emitNative) invokeTypex_WTypex_V(key typex.W, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WTypex_V) + return ret +} + +func (e *emitNative) invokeETTypex_WTypex_V(t typex.EventTime, key typex.W, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WTypex_W) + return ret +} + +func (e *emitNative) invokeTypex_WTypex_W(key typex.W, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WTypex_W) + return ret +} + +func (e *emitNative) invokeETTypex_WTypex_W(t typex.EventTime, key typex.W, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WTypex_X) + return ret +} + +func (e *emitNative) invokeTypex_WTypex_X(key typex.W, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WTypex_X) + return ret +} + +func (e *emitNative) invokeETTypex_WTypex_X(t typex.EventTime, key typex.W, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WTypex_Y) + return ret +} + +func (e *emitNative) invokeTypex_WTypex_Y(key typex.W, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WTypex_Y) + return ret +} + +func (e *emitNative) invokeETTypex_WTypex_Y(t typex.EventTime, key typex.W, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_WTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_WTypex_Z) + return ret +} + +func (e *emitNative) invokeTypex_WTypex_Z(key typex.W, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_WTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_WTypex_Z) + return ret +} + +func (e *emitNative) invokeETTypex_WTypex_Z(t typex.EventTime, key typex.W, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_X) + return ret +} + +func (e *emitNative) invokeTypex_X(elm typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_X) + return ret +} + +func (e *emitNative) invokeETTypex_X(t typex.EventTime, elm typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XByteSlice) + return ret +} + +func (e *emitNative) invokeTypex_XByteSlice(key typex.X, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XByteSlice) + return ret +} + +func (e *emitNative) invokeETTypex_XByteSlice(t typex.EventTime, key typex.X, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XBool) + return ret +} + +func (e *emitNative) invokeTypex_XBool(key typex.X, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XBool) + return ret +} + +func (e *emitNative) invokeETTypex_XBool(t typex.EventTime, key typex.X, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XString) + return ret +} + +func (e *emitNative) invokeTypex_XString(key typex.X, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XString) + return ret +} + +func (e *emitNative) invokeETTypex_XString(t typex.EventTime, key typex.X, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XInt) + return ret +} + +func (e *emitNative) invokeTypex_XInt(key typex.X, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XInt) + return ret +} + +func (e *emitNative) invokeETTypex_XInt(t typex.EventTime, key typex.X, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XInt8) + return ret +} + +func (e *emitNative) invokeTypex_XInt8(key typex.X, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XInt8) + return ret +} + +func (e *emitNative) invokeETTypex_XInt8(t typex.EventTime, key typex.X, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XInt16) + return ret +} + +func (e *emitNative) invokeTypex_XInt16(key typex.X, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XInt16) + return ret +} + +func (e *emitNative) invokeETTypex_XInt16(t typex.EventTime, key typex.X, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XInt32) + return ret +} + +func (e *emitNative) invokeTypex_XInt32(key typex.X, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XInt32) + return ret +} + +func (e *emitNative) invokeETTypex_XInt32(t typex.EventTime, key typex.X, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XInt64) + return ret +} + +func (e *emitNative) invokeTypex_XInt64(key typex.X, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XInt64) + return ret +} + +func (e *emitNative) invokeETTypex_XInt64(t typex.EventTime, key typex.X, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XUint) + return ret +} + +func (e *emitNative) invokeTypex_XUint(key typex.X, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XUint) + return ret +} + +func (e *emitNative) invokeETTypex_XUint(t typex.EventTime, key typex.X, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XUint8) + return ret +} + +func (e *emitNative) invokeTypex_XUint8(key typex.X, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XUint8) + return ret +} + +func (e *emitNative) invokeETTypex_XUint8(t typex.EventTime, key typex.X, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XUint16) + return ret +} + +func (e *emitNative) invokeTypex_XUint16(key typex.X, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XUint16) + return ret +} + +func (e *emitNative) invokeETTypex_XUint16(t typex.EventTime, key typex.X, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XUint32) + return ret +} + +func (e *emitNative) invokeTypex_XUint32(key typex.X, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XUint32) + return ret +} + +func (e *emitNative) invokeETTypex_XUint32(t typex.EventTime, key typex.X, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XUint64) + return ret +} + +func (e *emitNative) invokeTypex_XUint64(key typex.X, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XUint64) + return ret +} + +func (e *emitNative) invokeETTypex_XUint64(t typex.EventTime, key typex.X, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XFloat32) + return ret +} + +func (e *emitNative) invokeTypex_XFloat32(key typex.X, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XFloat32) + return ret +} + +func (e *emitNative) invokeETTypex_XFloat32(t typex.EventTime, key typex.X, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XFloat64) + return ret +} + +func (e *emitNative) invokeTypex_XFloat64(key typex.X, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XFloat64) + return ret +} + +func (e *emitNative) invokeETTypex_XFloat64(t typex.EventTime, key typex.X, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XTypex_T) + return ret +} + +func (e *emitNative) invokeTypex_XTypex_T(key typex.X, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XTypex_T) + return ret +} + +func (e *emitNative) invokeETTypex_XTypex_T(t typex.EventTime, key typex.X, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XTypex_U) + return ret +} + +func (e *emitNative) invokeTypex_XTypex_U(key typex.X, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XTypex_U) + return ret +} + +func (e *emitNative) invokeETTypex_XTypex_U(t typex.EventTime, key typex.X, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XTypex_V) + return ret +} + +func (e *emitNative) invokeTypex_XTypex_V(key typex.X, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XTypex_V) + return ret +} + +func (e *emitNative) invokeETTypex_XTypex_V(t typex.EventTime, key typex.X, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XTypex_W) + return ret +} + +func (e *emitNative) invokeTypex_XTypex_W(key typex.X, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XTypex_W) + return ret +} + +func (e *emitNative) invokeETTypex_XTypex_W(t typex.EventTime, key typex.X, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XTypex_X) + return ret +} + +func (e *emitNative) invokeTypex_XTypex_X(key typex.X, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XTypex_X) + return ret +} + +func (e *emitNative) invokeETTypex_XTypex_X(t typex.EventTime, key typex.X, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XTypex_Y) + return ret +} + +func (e *emitNative) invokeTypex_XTypex_Y(key typex.X, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XTypex_Y) + return ret +} + +func (e *emitNative) invokeETTypex_XTypex_Y(t typex.EventTime, key typex.X, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_XTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_XTypex_Z) + return ret +} + +func (e *emitNative) invokeTypex_XTypex_Z(key typex.X, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_XTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_XTypex_Z) + return ret +} + +func (e *emitNative) invokeETTypex_XTypex_Z(t typex.EventTime, key typex.X, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_Y) + return ret +} + +func (e *emitNative) invokeTypex_Y(elm typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_Y) + return ret +} + +func (e *emitNative) invokeETTypex_Y(t typex.EventTime, elm typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YByteSlice) + return ret +} + +func (e *emitNative) invokeTypex_YByteSlice(key typex.Y, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YByteSlice) + return ret +} + +func (e *emitNative) invokeETTypex_YByteSlice(t typex.EventTime, key typex.Y, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YBool) + return ret +} + +func (e *emitNative) invokeTypex_YBool(key typex.Y, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YBool) + return ret +} + +func (e *emitNative) invokeETTypex_YBool(t typex.EventTime, key typex.Y, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YString) + return ret +} + +func (e *emitNative) invokeTypex_YString(key typex.Y, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YString) + return ret +} + +func (e *emitNative) invokeETTypex_YString(t typex.EventTime, key typex.Y, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YInt) + return ret +} + +func (e *emitNative) invokeTypex_YInt(key typex.Y, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YInt) + return ret +} + +func (e *emitNative) invokeETTypex_YInt(t typex.EventTime, key typex.Y, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YInt8) + return ret +} + +func (e *emitNative) invokeTypex_YInt8(key typex.Y, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YInt8) + return ret +} + +func (e *emitNative) invokeETTypex_YInt8(t typex.EventTime, key typex.Y, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YInt16) + return ret +} + +func (e *emitNative) invokeTypex_YInt16(key typex.Y, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YInt16) + return ret +} + +func (e *emitNative) invokeETTypex_YInt16(t typex.EventTime, key typex.Y, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YInt32) + return ret +} + +func (e *emitNative) invokeTypex_YInt32(key typex.Y, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YInt32) + return ret +} + +func (e *emitNative) invokeETTypex_YInt32(t typex.EventTime, key typex.Y, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YInt64) + return ret +} + +func (e *emitNative) invokeTypex_YInt64(key typex.Y, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YInt64) + return ret +} + +func (e *emitNative) invokeETTypex_YInt64(t typex.EventTime, key typex.Y, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YUint) + return ret +} + +func (e *emitNative) invokeTypex_YUint(key typex.Y, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YUint) + return ret +} + +func (e *emitNative) invokeETTypex_YUint(t typex.EventTime, key typex.Y, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YUint8) + return ret +} + +func (e *emitNative) invokeTypex_YUint8(key typex.Y, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YUint8) + return ret +} + +func (e *emitNative) invokeETTypex_YUint8(t typex.EventTime, key typex.Y, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YUint16) + return ret +} + +func (e *emitNative) invokeTypex_YUint16(key typex.Y, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YUint16) + return ret +} + +func (e *emitNative) invokeETTypex_YUint16(t typex.EventTime, key typex.Y, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YUint32) + return ret +} + +func (e *emitNative) invokeTypex_YUint32(key typex.Y, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YUint32) + return ret +} + +func (e *emitNative) invokeETTypex_YUint32(t typex.EventTime, key typex.Y, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YUint64) + return ret +} + +func (e *emitNative) invokeTypex_YUint64(key typex.Y, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YUint64) + return ret +} + +func (e *emitNative) invokeETTypex_YUint64(t typex.EventTime, key typex.Y, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YFloat32) + return ret +} + +func (e *emitNative) invokeTypex_YFloat32(key typex.Y, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YFloat32) + return ret +} + +func (e *emitNative) invokeETTypex_YFloat32(t typex.EventTime, key typex.Y, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YFloat64) + return ret +} + +func (e *emitNative) invokeTypex_YFloat64(key typex.Y, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YFloat64) + return ret +} + +func (e *emitNative) invokeETTypex_YFloat64(t typex.EventTime, key typex.Y, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YTypex_T) + return ret +} + +func (e *emitNative) invokeTypex_YTypex_T(key typex.Y, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YTypex_T) + return ret +} + +func (e *emitNative) invokeETTypex_YTypex_T(t typex.EventTime, key typex.Y, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YTypex_U) + return ret +} + +func (e *emitNative) invokeTypex_YTypex_U(key typex.Y, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YTypex_U) + return ret +} + +func (e *emitNative) invokeETTypex_YTypex_U(t typex.EventTime, key typex.Y, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YTypex_V) + return ret +} + +func (e *emitNative) invokeTypex_YTypex_V(key typex.Y, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YTypex_V) + return ret +} + +func (e *emitNative) invokeETTypex_YTypex_V(t typex.EventTime, key typex.Y, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YTypex_W) + return ret +} + +func (e *emitNative) invokeTypex_YTypex_W(key typex.Y, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YTypex_W) + return ret +} + +func (e *emitNative) invokeETTypex_YTypex_W(t typex.EventTime, key typex.Y, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YTypex_X) + return ret +} + +func (e *emitNative) invokeTypex_YTypex_X(key typex.Y, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YTypex_X) + return ret +} + +func (e *emitNative) invokeETTypex_YTypex_X(t typex.EventTime, key typex.Y, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YTypex_Y) + return ret +} + +func (e *emitNative) invokeTypex_YTypex_Y(key typex.Y, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YTypex_Y) + return ret +} + +func (e *emitNative) invokeETTypex_YTypex_Y(t typex.EventTime, key typex.Y, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_YTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_YTypex_Z) + return ret +} + +func (e *emitNative) invokeTypex_YTypex_Z(key typex.Y, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_YTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_YTypex_Z) + return ret +} + +func (e *emitNative) invokeETTypex_YTypex_Z(t typex.EventTime, key typex.Y, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_Z) + return ret +} + +func (e *emitNative) invokeTypex_Z(elm typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_Z) + return ret +} + +func (e *emitNative) invokeETTypex_Z(t typex.EventTime, elm typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZByteSlice) + return ret +} + +func (e *emitNative) invokeTypex_ZByteSlice(key typex.Z, val []byte) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZByteSlice(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZByteSlice) + return ret +} + +func (e *emitNative) invokeETTypex_ZByteSlice(t typex.EventTime, key typex.Z, val []byte) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZBool) + return ret +} + +func (e *emitNative) invokeTypex_ZBool(key typex.Z, val bool) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZBool(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZBool) + return ret +} + +func (e *emitNative) invokeETTypex_ZBool(t typex.EventTime, key typex.Z, val bool) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZString) + return ret +} + +func (e *emitNative) invokeTypex_ZString(key typex.Z, val string) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZString(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZString) + return ret +} + +func (e *emitNative) invokeETTypex_ZString(t typex.EventTime, key typex.Z, val string) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZInt) + return ret +} + +func (e *emitNative) invokeTypex_ZInt(key typex.Z, val int) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZInt(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZInt) + return ret +} + +func (e *emitNative) invokeETTypex_ZInt(t typex.EventTime, key typex.Z, val int) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZInt8) + return ret +} + +func (e *emitNative) invokeTypex_ZInt8(key typex.Z, val int8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZInt8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZInt8) + return ret +} + +func (e *emitNative) invokeETTypex_ZInt8(t typex.EventTime, key typex.Z, val int8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZInt16) + return ret +} + +func (e *emitNative) invokeTypex_ZInt16(key typex.Z, val int16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZInt16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZInt16) + return ret +} + +func (e *emitNative) invokeETTypex_ZInt16(t typex.EventTime, key typex.Z, val int16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZInt32) + return ret +} + +func (e *emitNative) invokeTypex_ZInt32(key typex.Z, val int32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZInt32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZInt32) + return ret +} + +func (e *emitNative) invokeETTypex_ZInt32(t typex.EventTime, key typex.Z, val int32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZInt64) + return ret +} + +func (e *emitNative) invokeTypex_ZInt64(key typex.Z, val int64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZInt64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZInt64) + return ret +} + +func (e *emitNative) invokeETTypex_ZInt64(t typex.EventTime, key typex.Z, val int64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZUint) + return ret +} + +func (e *emitNative) invokeTypex_ZUint(key typex.Z, val uint) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZUint(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZUint) + return ret +} + +func (e *emitNative) invokeETTypex_ZUint(t typex.EventTime, key typex.Z, val uint) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZUint8) + return ret +} + +func (e *emitNative) invokeTypex_ZUint8(key typex.Z, val uint8) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZUint8(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZUint8) + return ret +} + +func (e *emitNative) invokeETTypex_ZUint8(t typex.EventTime, key typex.Z, val uint8) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZUint16) + return ret +} + +func (e *emitNative) invokeTypex_ZUint16(key typex.Z, val uint16) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZUint16(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZUint16) + return ret +} + +func (e *emitNative) invokeETTypex_ZUint16(t typex.EventTime, key typex.Z, val uint16) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZUint32) + return ret +} + +func (e *emitNative) invokeTypex_ZUint32(key typex.Z, val uint32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZUint32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZUint32) + return ret +} + +func (e *emitNative) invokeETTypex_ZUint32(t typex.EventTime, key typex.Z, val uint32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZUint64) + return ret +} + +func (e *emitNative) invokeTypex_ZUint64(key typex.Z, val uint64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZUint64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZUint64) + return ret +} + +func (e *emitNative) invokeETTypex_ZUint64(t typex.EventTime, key typex.Z, val uint64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZFloat32) + return ret +} + +func (e *emitNative) invokeTypex_ZFloat32(key typex.Z, val float32) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZFloat32(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZFloat32) + return ret +} + +func (e *emitNative) invokeETTypex_ZFloat32(t typex.EventTime, key typex.Z, val float32) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZFloat64) + return ret +} + +func (e *emitNative) invokeTypex_ZFloat64(key typex.Z, val float64) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZFloat64(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZFloat64) + return ret +} + +func (e *emitNative) invokeETTypex_ZFloat64(t typex.EventTime, key typex.Z, val float64) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZTypex_T) + return ret +} + +func (e *emitNative) invokeTypex_ZTypex_T(key typex.Z, val typex.T) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZTypex_T(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZTypex_T) + return ret +} + +func (e *emitNative) invokeETTypex_ZTypex_T(t typex.EventTime, key typex.Z, val typex.T) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZTypex_U) + return ret +} + +func (e *emitNative) invokeTypex_ZTypex_U(key typex.Z, val typex.U) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZTypex_U(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZTypex_U) + return ret +} + +func (e *emitNative) invokeETTypex_ZTypex_U(t typex.EventTime, key typex.Z, val typex.U) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZTypex_V) + return ret +} + +func (e *emitNative) invokeTypex_ZTypex_V(key typex.Z, val typex.V) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZTypex_V(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZTypex_V) + return ret +} + +func (e *emitNative) invokeETTypex_ZTypex_V(t typex.EventTime, key typex.Z, val typex.V) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZTypex_W) + return ret +} + +func (e *emitNative) invokeTypex_ZTypex_W(key typex.Z, val typex.W) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZTypex_W(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZTypex_W) + return ret +} + +func (e *emitNative) invokeETTypex_ZTypex_W(t typex.EventTime, key typex.Z, val typex.W) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZTypex_X) + return ret +} + +func (e *emitNative) invokeTypex_ZTypex_X(key typex.Z, val typex.X) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZTypex_X(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZTypex_X) + return ret +} + +func (e *emitNative) invokeETTypex_ZTypex_X(t typex.EventTime, key typex.Z, val typex.X) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZTypex_Y) + return ret +} + +func (e *emitNative) invokeTypex_ZTypex_Y(key typex.Z, val typex.Y) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZTypex_Y(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZTypex_Y) + return ret +} + +func (e *emitNative) invokeETTypex_ZTypex_Y(t typex.EventTime, key typex.Z, val typex.Y) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerTypex_ZTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeTypex_ZTypex_Z) + return ret +} + +func (e *emitNative) invokeTypex_ZTypex_Z(key typex.Z, val typex.Z) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerETTypex_ZTypex_Z(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeETTypex_ZTypex_Z) + return ret +} + +func (e *emitNative) invokeETTypex_ZTypex_Z(t typex.EventTime, key typex.Z, val typex.Z) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val)} + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} diff --git a/sdks/go/pkg/beam/core/runtime/exec/emitters.tmpl b/sdks/go/pkg/beam/core/runtime/exec/emitters.tmpl new file mode 100644 index 00000000000..1d7300bce6b --- /dev/null +++ b/sdks/go/pkg/beam/core/runtime/exec/emitters.tmpl @@ -0,0 +1,108 @@ +// 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 ( + "context" + "reflect" + + "github.com/apache/beam/sdks/go/pkg/beam/core/typex" +) + +func init() { +{{- range $x := .X}} + RegisterEmitter(reflect.TypeOf((*func ({{$x.Type}}))(nil)).Elem(), emitMaker{{$x.Name}}) + RegisterEmitter(reflect.TypeOf((*func (typex.EventTime, {{$x.Type}}))(nil)).Elem(), emitMakerET{{$x.Name}}) +{{- range $y := .Y}} + RegisterEmitter(reflect.TypeOf((*func ({{$x.Type}}, {{$y.Type}}))(nil)).Elem(), emitMaker{{$x.Name}}{{$y.Name}}) + RegisterEmitter(reflect.TypeOf((*func (typex.EventTime, {{$x.Type}}, {{$y.Type}}))(nil)).Elem(), emitMakerET{{$x.Name}}{{$y.Name}}) +{{- end}} +{{- end}} +} + +type emitNative struct { + n ElementProcessor + fn reflect.Value + + ctx context.Context + et typex.EventTime +} + +func (e *emitNative) Init(ctx context.Context, et typex.EventTime) error { + e.ctx = ctx + e.et = et + return nil +} + +func (e *emitNative) Value() reflect.Value { + return e.fn +} + +{{range $x := .X}} +func emitMaker{{$x.Name}}(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invoke{{$x.Name}}) + return ret +} + +func (e *emitNative) invoke{{$x.Name}}(elm {{$x.Type}}) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(elm) } + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerET{{$x.Name}}(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeET{{$x.Name}}) + return ret +} + +func (e *emitNative) invokeET{{$x.Name}}(t typex.EventTime, elm {{$x.Type}}) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(elm) } + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +{{range $y := .Y}} +func emitMaker{{$x.Name}}{{$y.Name}}(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invoke{{$x.Name}}{{$y.Name}}) + return ret +} + +func (e *emitNative) invoke{{$x.Name}}{{$y.Name}}(key {{$x.Type}}, val {{$y.Type}}) { + value := FullValue{Timestamp: e.et, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val) } + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} + +func emitMakerET{{$x.Name}}{{$y.Name}}(n ElementProcessor) ReusableEmitter { + ret := &emitNative{n: n} + ret.fn = reflect.ValueOf(ret.invokeET{{$x.Name}}{{$y.Name}}) + return ret +} + +func (e *emitNative) invokeET{{$x.Name}}{{$y.Name}}(t typex.EventTime, key {{$x.Type}}, val {{$y.Type}}) { + value := FullValue{Timestamp: t, Elm: reflect.ValueOf(key), Elm2: reflect.ValueOf(val) } + if err := e.n.ProcessElement(e.ctx, value); err != nil { + panic(err) + } +} +{{end}} +{{end}} diff --git a/sdks/go/pkg/beam/core/runtime/exec/fn.go b/sdks/go/pkg/beam/core/runtime/exec/fn.go index 2897f4ad16f..1e5ce6b3f4e 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/fn.go +++ b/sdks/go/pkg/beam/core/runtime/exec/fn.go @@ -332,70 +332,3 @@ func (v *fixedValue) Value() reflect.Value { func (v *fixedValue) Reset() error { return nil } - -// ReusableEmitter is a resettable value needed to hold the implicit context and -// emit event time. -type ReusableEmitter interface { - // Init resets the value. Can be called multiple times. - Init(ctx context.Context, t typex.EventTime) error - // Value returns the side input value. Contant value. - Value() reflect.Value -} - -type emitValue struct { - n Node - fn reflect.Value - types []reflect.Type - - ctx context.Context - et typex.EventTime -} - -func makeEmit(t reflect.Type, n Node) ReusableEmitter { - types, ok := funcx.UnfoldEmit(t) - if !ok { - panic(fmt.Sprintf("illegal emit type: %v", t)) - } - - ret := &emitValue{n: n, types: types} - ret.fn = reflect.MakeFunc(t, ret.invoke) - return ret -} - -func (e *emitValue) Init(ctx context.Context, et typex.EventTime) error { - e.ctx = ctx - e.et = et - return nil -} - -func (e *emitValue) Value() reflect.Value { - return e.fn -} - -func (e *emitValue) invoke(args []reflect.Value) []reflect.Value { - value := FullValue{Timestamp: e.et} - isKey := true - for i, t := range e.types { - switch { - case t == typex.EventTimeType: - value.Timestamp = args[i].Interface().(typex.EventTime) - case isKey: - value.Elm = args[i] - isKey = false - default: - value.Elm2 = args[i] - } - } - - if err := e.n.ProcessElement(e.ctx, value); err != nil { - // NOTE(herohde) 12/11/2017: emitters do not return an error, so if there - // are problems we rely on the receiving node capturing the error. - // Furthermore, we panic to quickly halt processing -- a corner-case - // is that this panic unwinds _through_ user code and may be caught or - // ignored, in which case we fall back failing bundle when the error is - // returned by FinishBundle. - - panic(err) - } - return nil -} diff --git a/sdks/go/pkg/beam/core/runtime/exec/fn_test.go b/sdks/go/pkg/beam/core/runtime/exec/fn_test.go index 0cb8f36b319..16396908f5d 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/fn_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/fn_test.go @@ -121,3 +121,143 @@ func makeArgs(list []interface{}) []reflect.Value { } return ret } + +// Benchmarks + +// NOTE(herohde) 12/19/2017: example run on a laptop +// +// BenchmarkDirectCall-4 2000000000 0.26 ns/op +// BenchmarkIndirectCall-4 1000000000 2.28 ns/op +// BenchmarkReflectedAndBackCall-4 1000000000 2.40 ns/op +// BenchmarkReflectCall-4 10000000 197 ns/op +// BenchmarkReflectCallNewArgs-4 10000000 200 ns/op +// BenchmarkReflectCallReuseArgs-4 10000000 196 ns/op +// BenchmarkInvokeCall-4 3000000 452 ns/op +// BenchmarkInvokeCallExtra-4 3000000 419 ns/op +// BenchmarkReflectFnCall-4 5000000 340 ns/op +// BenchmarkInvokeFnCall-4 2000000 604 ns/op +// BenchmarkInvokeFnCallExtra-4 3000000 559 ns/op + +func inc(n int) int { + return n + 1 +} + +func BenchmarkDirectCall(b *testing.B) { + n := 0 + for i := 0; i < b.N; i++ { + n = inc(n) + } + b.Log(n) +} + +func BenchmarkIndirectCall(b *testing.B) { + fn := func(int) int { return 0 } + if b.N > 0 { + fn = inc // this is always set, but the compiler doesn't know + } + + n := 0 + for i := 0; i < b.N; i++ { + n = fn(n) + } + b.Log(n) +} + +func BenchmarkReflectedAndBackCall(b *testing.B) { + fn := reflect.ValueOf(inc).Interface().(func(int) int) + n := 0 + for i := 0; i < b.N; i++ { + n = fn(n) + } + b.Log(n) +} + +func BenchmarkReflectCall(b *testing.B) { + fn := reflect.ValueOf(inc) + n := reflect.ValueOf(0) + for i := 0; i < b.N; i++ { + n = fn.Call([]reflect.Value{n})[0] + } + b.Log(n.Interface()) +} + +func BenchmarkReflectCallNewArgs(b *testing.B) { + fn := reflect.ValueOf(inc) + n := reflect.ValueOf(0) + for i := 0; i < b.N; i++ { + args := make([]reflect.Value, 1, 1) + args[0] = n + n = fn.Call(args)[0] + } + b.Log(n.Interface()) +} + +func BenchmarkReflectCallReuseArgs(b *testing.B) { + fn := reflect.ValueOf(inc) + n := reflect.ValueOf(0) + args := make([]reflect.Value, 1, 1) + for i := 0; i < b.N; i++ { + args[0] = n + n = fn.Call(args)[0] + } + b.Log(n.Interface()) +} + +func BenchmarkInvokeCall(b *testing.B) { + fn, _ := funcx.New(inc) + ctx := context.Background() + n := reflect.ValueOf(0) + for i := 0; i < b.N; i++ { + ret, _ := Invoke(ctx, fn, &MainInput{Key: FullValue{Elm: n}}) + n = ret.Elm + } + b.Log(n.Interface()) +} + +func BenchmarkInvokeCallExtra(b *testing.B) { + fn, _ := funcx.New(inc) + ctx := context.Background() + n := reflect.ValueOf(0) + for i := 0; i < b.N; i++ { + ret, _ := Invoke(ctx, fn, nil, n) + n = ret.Elm + } + b.Log(n.Interface()) +} + +// The below take the additional overhead of MakeFunc. + +func incFn(args []reflect.Value) []reflect.Value { + return []reflect.Value{reflect.ValueOf(args[0].Interface().(int) + 1)} +} + +func BenchmarkReflectFnCall(b *testing.B) { + fn := reflect.MakeFunc(reflect.TypeOf(inc), incFn) + n := reflect.ValueOf(0) + for i := 0; i < b.N; i++ { + n = fn.Call([]reflect.Value{n})[0] + } + b.Log(n.Interface()) +} + +func BenchmarkInvokeFnCall(b *testing.B) { + fn, _ := funcx.New(reflect.MakeFunc(reflect.TypeOf(inc), incFn).Interface().(func(int) int)) + ctx := context.Background() + n := reflect.ValueOf(0) + for i := 0; i < b.N; i++ { + ret, _ := Invoke(ctx, fn, &MainInput{Key: FullValue{Elm: n}}) + n = ret.Elm + } + b.Log(n.Interface()) +} + +func BenchmarkInvokeFnCallExtra(b *testing.B) { + fn, _ := funcx.New(reflect.MakeFunc(reflect.TypeOf(inc), incFn).Interface().(func(int) int)) + ctx := context.Background() + n := reflect.ValueOf(0) + for i := 0; i < b.N; i++ { + ret, _ := Invoke(ctx, fn, nil, n) + n = ret.Elm + } + b.Log(n.Interface()) +} diff --git a/sdks/go/pkg/beam/core/runtime/exec/unit.go b/sdks/go/pkg/beam/core/runtime/exec/unit.go index 158301773df..bea5469a08d 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/unit.go +++ b/sdks/go/pkg/beam/core/runtime/exec/unit.go @@ -60,12 +60,18 @@ type Root interface { Process(ctx context.Context) error } -// Node represents an single-bundle processing unit. Each node contains -// its processing continuation, notably other nodes. -type Node interface { - Unit +// TODO(herohde) 12/21/2017: maybe switch to interface{} from reflect.Value. +// ElementProcessor presents a component that can process an element. +type ElementProcessor interface { // Call processes a single element. If GBK or CoGBK result, the values // are populated. Otherwise, they're empty. ProcessElement(ctx context.Context, elm FullValue, values ...ReStream) error } + +// Node represents an single-bundle processing unit. Each node contains +// its processing continuation, notably other nodes. +type Node interface { + Unit + ElementProcessor +} ---------------------------------------------------------------- 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)